From cc95c27cd3965776d96df5fbb7937ab159715a35 Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Fri, 25 Sep 2015 15:25:29 -0700 Subject: [PATCH 001/331] Ignore if the partitions have no corresponfing hdfs valid path. --- .../java/com/facebook/presto/hive/util/HiveFileIterator.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/util/HiveFileIterator.java b/presto-hive/src/main/java/com/facebook/presto/hive/util/HiveFileIterator.java index 160f1dc0e964..077711b17982 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/util/HiveFileIterator.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/util/HiveFileIterator.java @@ -33,7 +33,6 @@ import java.util.Properties; import static com.facebook.presto.hive.HiveErrorCode.HIVE_FILESYSTEM_ERROR; -import static com.facebook.presto.hive.HiveErrorCode.HIVE_FILE_NOT_FOUND; import static java.util.Objects.requireNonNull; public class HiveFileIterator @@ -94,7 +93,8 @@ protected LocatedFileStatus computeNext() return endOfData(); } catch (FileNotFoundException e) { - throw new PrestoException(HIVE_FILE_NOT_FOUND, "Partition location does not exist: " + path); + // We are okay if the path does not exist. + return endOfData(); } catch (IOException e) { throw new PrestoException(HIVE_FILESYSTEM_ERROR, e); From bce8187ec60304f2f30262ab710b571885b3bdb9 Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Thu, 1 Oct 2015 10:40:07 -0700 Subject: [PATCH 002/331] Append nulls for missing values in Parquet. Parquet only calls converts for which it found the values. The missing values are not reported. The BlockBuilder must be appended with nulls for the missing values based on fieldIndex of the currently read value by Parquet. --- .../hive/parquet/ParquetHiveRecordCursor.java | 71 +++++++++++++------ 1 file changed, 50 insertions(+), 21 deletions(-) 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 8e9cf0290ff8..c7d8c0c7901f 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 @@ -446,7 +446,7 @@ public PrestoReadSupport(boolean useParquetColumnNames, List c converters.add(new ParquetPrimitiveColumnConverter(i)); } else { - converters.add(new ParquetColumnConverter(createGroupConverter(types[i], parquetType.getName(), parquetType), i)); + converters.add(new ParquetColumnConverter(createGroupConverter(types[i], parquetType.getName(), parquetType, i), i)); } } } @@ -674,25 +674,25 @@ private abstract static class GroupedConverter public abstract Block getBlock(); } - private static BlockConverter createConverter(Type prestoType, String columnName, parquet.schema.Type parquetType) + private static BlockConverter createConverter(Type prestoType, String columnName, parquet.schema.Type parquetType, int fieldIndex) { if (parquetType.isPrimitive()) { - return new ParquetPrimitiveConverter(prestoType); + return new ParquetPrimitiveConverter(prestoType, fieldIndex); } - return createGroupConverter(prestoType, columnName, parquetType); + return createGroupConverter(prestoType, columnName, parquetType, fieldIndex); } - private static GroupedConverter createGroupConverter(Type prestoType, String columnName, parquet.schema.Type parquetType) + private static GroupedConverter createGroupConverter(Type prestoType, String columnName, parquet.schema.Type parquetType, int fieldIndex) { GroupType groupType = parquetType.asGroupType(); switch (prestoType.getTypeSignature().getBase()) { case ARRAY: - return new ParquetListConverter(prestoType, columnName, groupType); + return new ParquetListConverter(prestoType, columnName, groupType, fieldIndex); case MAP: - return new ParquetMapConverter(prestoType, columnName, groupType); + return new ParquetMapConverter(prestoType, columnName, groupType, fieldIndex); case ROW: - return new ParquetStructConverter(prestoType, columnName, groupType); + return new ParquetStructConverter(prestoType, columnName, groupType, fieldIndex); default: throw new IllegalArgumentException("Column " + columnName + " type " + parquetType.getOriginalType() + " not supported"); } @@ -705,13 +705,14 @@ private static class ParquetStructConverter private static final int NULL_BUILDER_SIZE_IN_BYTES_THRESHOLD = 32768; private final Type rowType; + private final int fieldIndex; private final List converters; private BlockBuilder builder; private BlockBuilder nullBuilder; // used internally when builder is set to null private BlockBuilder currentEntryBuilder; - public ParquetStructConverter(Type prestoType, String columnName, GroupType entryType) + public ParquetStructConverter(Type prestoType, String columnName, GroupType entryType, int fieldIndex) { checkArgument(ROW.equals(prestoType.getTypeSignature().getBase())); List prestoTypeParameters = prestoType.getTypeParameters(); @@ -719,11 +720,12 @@ public ParquetStructConverter(Type prestoType, String columnName, GroupType entr checkArgument(prestoTypeParameters.size() == fieldTypes.size()); this.rowType = prestoType; + this.fieldIndex = fieldIndex; ImmutableList.Builder converters = ImmutableList.builder(); for (int i = 0; i < prestoTypeParameters.size(); i++) { parquet.schema.Type fieldType = fieldTypes.get(i); - converters.add(createConverter(prestoTypeParameters.get(i), columnName + "." + fieldType.getName(), fieldType)); + converters.add(createConverter(prestoTypeParameters.get(i), columnName + "." + fieldType.getName(), fieldType, i)); } this.converters = converters.build(); } @@ -750,6 +752,9 @@ public void start() currentEntryBuilder = nullBuilder.beginBlockEntry(); } else { + while (builder.getPositionCount() < fieldIndex) { + builder.appendNull(); + } currentEntryBuilder = builder.beginBlockEntry(); } for (BlockConverter converter : converters) { @@ -763,6 +768,10 @@ public void end() for (BlockConverter converter : converters) { converter.afterValue(); } + while (currentEntryBuilder.getPositionCount() < converters.size()) { + currentEntryBuilder.appendNull(); + } + if (builder == null) { nullBuilder.closeEntry(); } @@ -791,13 +800,14 @@ private static class ParquetListConverter private static final int NULL_BUILDER_SIZE_IN_BYTES_THRESHOLD = 32768; private final Type arrayType; + private final int fieldIndex; private final BlockConverter elementConverter; private BlockBuilder builder; private BlockBuilder nullBuilder; // used internally when builder is set to null private BlockBuilder currentEntryBuilder; - public ParquetListConverter(Type prestoType, String columnName, GroupType listType) + public ParquetListConverter(Type prestoType, String columnName, GroupType listType, int fieldIndex) { checkArgument(listType.getFieldCount() == 1, "Expected LIST column '%s' to only have one field, but has %s fields", @@ -806,6 +816,7 @@ public ParquetListConverter(Type prestoType, String columnName, GroupType listTy checkArgument(ARRAY.equals(prestoType.getTypeSignature().getBase())); this.arrayType = prestoType; + this.fieldIndex = fieldIndex; // The Parquet specification requires that the element value of a // LIST type be wrapped in an inner repeated group, like so: @@ -821,7 +832,7 @@ public ParquetListConverter(Type prestoType, String columnName, GroupType listTy // documentation at http://git.io/vOpNz. parquet.schema.Type elementType = listType.getType(0); if (isElementType(elementType, listType.getName())) { - elementConverter = createConverter(prestoType.getTypeParameters().get(0), columnName + ".element", elementType); + elementConverter = createConverter(prestoType.getTypeParameters().get(0), columnName + ".element", elementType, 0); } else { elementConverter = new ParquetListEntryConverter(prestoType.getTypeParameters().get(0), columnName, elementType.asGroupType()); @@ -875,6 +886,9 @@ public void start() currentEntryBuilder = nullBuilder.beginBlockEntry(); } else { + while (builder.getPositionCount() < fieldIndex) { + builder.appendNull(); + } currentEntryBuilder = builder.beginBlockEntry(); } elementConverter.beforeValue(currentEntryBuilder); @@ -926,7 +940,7 @@ public ParquetListEntryConverter(Type prestoType, String columnName, GroupType e columnName, elementType.getFieldCount()); - elementConverter = createConverter(prestoType, columnName + ".element", elementType.getType(0)); + elementConverter = createConverter(prestoType, columnName + ".element", elementType.getType(0), 0); } @Override @@ -969,13 +983,14 @@ private static class ParquetMapConverter private static final int NULL_BUILDER_SIZE_IN_BYTES_THRESHOLD = 32768; private final Type mapType; + private final int fieldIndex; private final ParquetMapEntryConverter entryConverter; private BlockBuilder builder; private BlockBuilder nullBuilder; // used internally when builder is set to null private BlockBuilder currentEntryBuilder; - public ParquetMapConverter(Type type, String columnName, GroupType mapType) + public ParquetMapConverter(Type type, String columnName, GroupType mapType, int fieldIndex) { checkArgument(mapType.getFieldCount() == 1, "Expected MAP column '%s' to only have one field, but has %s fields", @@ -983,6 +998,7 @@ public ParquetMapConverter(Type type, String columnName, GroupType mapType) mapType.getFieldCount()); this.mapType = type; + this.fieldIndex = fieldIndex; parquet.schema.Type entryType = mapType.getFields().get(0); @@ -1014,6 +1030,9 @@ public void start() currentEntryBuilder = nullBuilder.beginBlockEntry(); } else { + while (builder.getPositionCount() < fieldIndex) { + builder.appendNull(); + } currentEntryBuilder = builder.beginBlockEntry(); } entryConverter.beforeValue(currentEntryBuilder); @@ -1084,8 +1103,8 @@ public ParquetMapEntryConverter(Type prestoType, String columnName, GroupType en columnName, entryGroupType.getType(0)); - keyConverter = createConverter(prestoType.getTypeParameters().get(0), columnName + ".key", entryGroupType.getFields().get(0)); - valueConverter = createConverter(prestoType.getTypeParameters().get(1), columnName + ".value", entryGroupType.getFields().get(1)); + keyConverter = createConverter(prestoType.getTypeParameters().get(0), columnName + ".key", entryGroupType.getFields().get(0), 0); + valueConverter = createConverter(prestoType.getTypeParameters().get(1), columnName + ".value", entryGroupType.getFields().get(1), 1); } @Override @@ -1131,12 +1150,14 @@ private static class ParquetPrimitiveConverter implements BlockConverter { private final Type type; + private final int fieldIndex; private BlockBuilder builder; private boolean wroteValue; - public ParquetPrimitiveConverter(Type type) + public ParquetPrimitiveConverter(Type type, int fieldIndex) { this.type = type; + this.fieldIndex = fieldIndex; } @Override @@ -1149,11 +1170,13 @@ public void beforeValue(BlockBuilder builder) @Override public void afterValue() { - if (wroteValue) { - return; - } + } - builder.appendNull(); + private void addMissingValues() + { + while (builder.getPositionCount() < fieldIndex) { + builder.appendNull(); + } } @Override @@ -1187,6 +1210,7 @@ public void addValueFromDictionary(int dictionaryId) @Override public void addBoolean(boolean value) { + addMissingValues(); BOOLEAN.writeBoolean(builder, value); wroteValue = true; } @@ -1194,6 +1218,7 @@ public void addBoolean(boolean value) @Override public void addDouble(double value) { + addMissingValues(); DOUBLE.writeDouble(builder, value); wroteValue = true; } @@ -1201,6 +1226,7 @@ public void addDouble(double value) @Override public void addLong(long value) { + addMissingValues(); BIGINT.writeLong(builder, value); wroteValue = true; } @@ -1208,6 +1234,7 @@ public void addLong(long value) @Override public void addBinary(Binary value) { + addMissingValues(); if (type == TIMESTAMP) { builder.writeLong(ParquetTimestampUtils.getTimestampMillis(value)).closeEntry(); } @@ -1220,6 +1247,7 @@ public void addBinary(Binary value) @Override public void addFloat(float value) { + addMissingValues(); DOUBLE.writeDouble(builder, value); wroteValue = true; } @@ -1227,6 +1255,7 @@ public void addFloat(float value) @Override public void addInt(int value) { + addMissingValues(); BIGINT.writeLong(builder, value); wroteValue = true; } From 3bbcbb77181ef0d9a543dcbcceb988fcfad0de39 Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Thu, 1 Oct 2015 11:27:09 -0700 Subject: [PATCH 003/331] [maven-release-plugin] prepare for tw-0.1 release. --- pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 35 files changed, 35 insertions(+), 35 deletions(-) diff --git a/pom.xml b/pom.xml index 1ce3b8933afa..2e906d46ab60 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT pom presto-root diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index bf0ea89449f7..becc2aad6552 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index bea67680cd7e..fcc11ed9be7d 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index eea12d2ea187..3dca897d8d00 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 4bd015fe78bf..5f68c004c6e3 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index e4f4e3b4f115..b4276f9a2118 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 812de79ce39b..e0fa76890e65 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 53672dfabdb5..33caecd03adb 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 460610625774..22a19de7cdf6 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 8b44291c6a43..c805d506939e 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index cac75be155a5..4c764cc832c3 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 3caded1f3f89..f775951da3c0 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 01deea780ac4..28bba6912311 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 9028adcbcaf7..0b47093c5d7a 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index a0edca6dd537..40bfd35c4cd0 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 9f6939498365..198b7dba9b74 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 4dbffd33d13f..11364ccb4163 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index bf3b0a07e618..727e8efad01c 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index a8b129bfcf72..f2a2d446e7d3 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 4f2b30229f50..d4678be9d7e6 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 604afe0f5e1f..3005cbee88c0 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 6fe0e61a7972..2d9f2539d18a 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index eef17295c589..f99df2a20914 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 755c12ab097f..2e3f97bf0bf4 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-postgresql diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 83d83e4accff..72ce96885268 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 25555b4a25d2..6d344eada7e4 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 8d44339c6c78..1dc2d16c7fb8 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 33e66de413fc..30e1a7c0a1e8 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index ac51dbc823c4..468278f4a955 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index ad8a095f4f39..2e9c6a0c9c13 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 40bafdb094d3..05fe47646434 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index a321b43f5771..d30e0605f888 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 44501fcdf2bf..9ec7a73cd2da 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index ea19dd90dcf6..932924f8b8ed 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 3491c458ba4d..8cade2a0ed97 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.120-SNAPSHOT + 0.119-tw-0.1-SNAPSHOT presto-verifier From ca8560fde9575f6d45dd6b2d24c5775105c5bf32 Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Thu, 1 Oct 2015 11:42:10 -0700 Subject: [PATCH 004/331] [maven-release-plugin] prepare for release 0.119-tw-0.1. --- pom.xml | 8 ++++---- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 35 files changed, 38 insertions(+), 38 deletions(-) diff --git a/pom.xml b/pom.xml index 2e906d46ab60..73f572f41e55 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 pom presto-root @@ -28,9 +28,9 @@ - scm:git:git://github.com/facebook/presto.git - https://github.com/facebook/presto - HEAD + scm:git:git://github.com/twitter-forks/presto.git + https://github.com/twitter-forks/presto + 0.119-tw-0.1 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index becc2aad6552..bf85b92c69ab 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index fcc11ed9be7d..dabd15b59a88 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 3dca897d8d00..7900d3a15d20 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 5f68c004c6e3..1d62047b7b8c 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index b4276f9a2118..85b7806119be 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index e0fa76890e65..53f55d5d5c91 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 33caecd03adb..242560f261a8 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 22a19de7cdf6..364b9e3e3825 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index c805d506939e..07d4930b6b8b 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 4c764cc832c3..1af8e55a0d46 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index f775951da3c0..fa858095ac23 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 28bba6912311..99226c90a798 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 0b47093c5d7a..b0a353cd0575 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 40bfd35c4cd0..b94aaa9676fd 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 198b7dba9b74..f4a2653f06a2 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 11364ccb4163..5c1c1418f45c 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 727e8efad01c..029add19febd 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index f2a2d446e7d3..5bdea99bb96a 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index d4678be9d7e6..911c908772bb 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 3005cbee88c0..21d1697a581b 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 2d9f2539d18a..5730006d2752 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index f99df2a20914..f14de7f6b725 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 2e3f97bf0bf4..a43d0b35f4a0 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-postgresql diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 72ce96885268..9a8940684940 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 6d344eada7e4..506e6e6fb4d8 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 1dc2d16c7fb8..da5609eb349d 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 30e1a7c0a1e8..2c9e1c9a8c72 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 468278f4a955..9c258c8b88dc 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 2e9c6a0c9c13..b41a6498d846 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 05fe47646434..1ee4d99cfda8 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index d30e0605f888..661f3db9a240 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 9ec7a73cd2da..f76a75aaba9b 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 932924f8b8ed..6b3161861e6f 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 8cade2a0ed97..c22f6a70eb88 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.119-tw-0.1-SNAPSHOT + 0.119-tw-0.1 presto-verifier From ec109d1dce3d3d93a9335f3750c8bd3c424bb22e Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Fri, 2 Oct 2015 15:47:29 -0700 Subject: [PATCH 005/331] Compare types ignoring cases. This is not complete solution and should be reverted once #2863 is resolved. --- .../java/com/facebook/presto/metadata/FunctionRegistry.java | 2 +- .../com/facebook/presto/operator/scalar/RowFieldReference.java | 3 +-- .../com/facebook/presto/sql/analyzer/ExpressionAnalyzer.java | 2 +- .../main/java/com/facebook/presto/type/RowParametricType.java | 2 +- .../main/java/com/facebook/presto/spi/type/TypeSignature.java | 2 +- 5 files changed, 5 insertions(+), 6 deletions(-) diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/FunctionRegistry.java b/presto-main/src/main/java/com/facebook/presto/metadata/FunctionRegistry.java index 8c4d9a48af1d..45bff811f1a1 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/FunctionRegistry.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/FunctionRegistry.java @@ -449,7 +449,7 @@ public FunctionInfo resolveFunction(QualifiedName name, List para RowType rowType = RowParametricType.ROW.createType(resolveTypes(typeSignature.getParameters(), typeManager), typeSignature.getLiteralParameters()); // search for exact match for (ParametricFunction function : RowParametricType.ROW.createFunctions(rowType)) { - if (!function.getSignature().getName().equals(name.toString())) { + if (!function.getSignature().getName().equalsIgnoreCase(name.toString())) { continue; } Map boundTypeParameters = function.getSignature().bindTypeParameters(resolvedTypes, false, typeManager); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowFieldReference.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowFieldReference.java index f4a94125f49a..4291e7970fc9 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowFieldReference.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/RowFieldReference.java @@ -28,7 +28,6 @@ import java.lang.invoke.MethodHandle; import java.util.Map; -import java.util.Optional; import static com.facebook.presto.sql.QueryUtil.mangleFieldReference; import static com.facebook.presto.type.RowType.RowField; @@ -59,7 +58,7 @@ public RowFieldReference(RowType type, String fieldName) Type returnType = null; int index = 0; for (RowField field : type.getFields()) { - if (field.getName().equals(Optional.of(fieldName))) { + if (field.getName().get().equalsIgnoreCase(fieldName)) { returnType = field.getType(); break; } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionAnalyzer.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionAnalyzer.java index 22b3d3559ec0..6239c559d708 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionAnalyzer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionAnalyzer.java @@ -318,7 +318,7 @@ private Type tryVisitRowFieldAccessor(QualifiedNameReference node) RowType rowType = checkType(field.getType(), RowType.class, "field.getType()"); Type rowFieldType = null; for (RowField rowField : rowType.getFields()) { - if (rowField.getName().equals(Optional.of(node.getName().getSuffix()))) { + if (rowField.getName().get().equalsIgnoreCase(node.getName().getSuffix())) { rowFieldType = rowField.getType(); break; } diff --git a/presto-main/src/main/java/com/facebook/presto/type/RowParametricType.java b/presto-main/src/main/java/com/facebook/presto/type/RowParametricType.java index 5a4804d99868..f175754d696a 100644 --- a/presto-main/src/main/java/com/facebook/presto/type/RowParametricType.java +++ b/presto-main/src/main/java/com/facebook/presto/type/RowParametricType.java @@ -65,7 +65,7 @@ public List createFunctions(Type type) ImmutableList.Builder builder = ImmutableList.builder(); for (RowField field : rowType.getFields()) { field.getName() - .ifPresent(name -> builder.add(new RowFieldReference(rowType, field.getName().get()))); + .ifPresent(name -> builder.add(new RowFieldReference(rowType, field.getName().get().toLowerCase()))); } return builder.build(); } diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/type/TypeSignature.java b/presto-spi/src/main/java/com/facebook/presto/spi/type/TypeSignature.java index 8e5d924fe399..75ffe4906f81 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/type/TypeSignature.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/type/TypeSignature.java @@ -189,7 +189,7 @@ private static Object parseLiteral(String literal) { if (literal.startsWith("'") || literal.endsWith("'")) { checkArgument(literal.startsWith("'") && literal.endsWith("'"), "Bad literal: '%s'", literal); - return literal.substring(1, literal.length() - 1); + return literal.substring(1, literal.length() - 1).toLowerCase(); } else { return Long.parseLong(literal); From c76b42e910bfbf989078171587dc6700f2757275 Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Fri, 9 Oct 2015 12:43:09 -0700 Subject: [PATCH 006/331] Revert "Reduce allocation of hadoop configuration objects" This reverts commit 7365f1ad3a75b478bcdecca65676d7857b843699. This blocked us from reading from mountable clusters. --- .../com/facebook/presto/hive/HiveHdfsConfiguration.java | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) 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 8ad5acc65c9f..dc6d0f46f3ed 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 @@ -18,25 +18,19 @@ import javax.inject.Inject; import java.net.URI; -import java.util.Map; import static java.util.Objects.requireNonNull; public class HiveHdfsConfiguration implements HdfsConfiguration { - private static final Configuration DEFAULT_CONFIGURATION = new Configuration(); - @SuppressWarnings("ThreadLocalNotStaticFinal") private final ThreadLocal hadoopConfiguration = new ThreadLocal() { @Override protected Configuration initialValue() { - Configuration config = new Configuration(false); - for (Map.Entry entry : DEFAULT_CONFIGURATION) { - config.set(entry.getKey(), entry.getValue()); - } + Configuration config = new Configuration(); updater.updateConfiguration(config); return config; } From f0d9ab0d9780de736d7c6893ccf4960a57dc8fba Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Tue, 13 Oct 2015 14:25:46 -0700 Subject: [PATCH 007/331] Bump up zookeeper version to 3.4.6 --- pom.xml | 10 +++++++++- .../facebook/presto/kafka/util/EmbeddedZookeeper.java | 7 ++++--- 2 files changed, 13 insertions(+), 4 deletions(-) diff --git a/pom.xml b/pom.xml index 301487adddb2..c845b238ec5b 100644 --- a/pom.xml +++ b/pom.xml @@ -624,7 +624,7 @@ org.apache.zookeeper zookeeper - 3.3.6 + 3.4.6 junit @@ -634,6 +634,14 @@ log4j log4j + + org.slf4j + slf4j-jdk14 + + + org.slf4j + slf4j-log4j12 + diff --git a/presto-kafka/src/test/java/com/facebook/presto/kafka/util/EmbeddedZookeeper.java b/presto-kafka/src/test/java/com/facebook/presto/kafka/util/EmbeddedZookeeper.java index fddf5c845746..0f319cee1cbc 100644 --- a/presto-kafka/src/test/java/com/facebook/presto/kafka/util/EmbeddedZookeeper.java +++ b/presto-kafka/src/test/java/com/facebook/presto/kafka/util/EmbeddedZookeeper.java @@ -14,7 +14,7 @@ package com.facebook.presto.kafka.util; import com.google.common.io.Files; -import org.apache.zookeeper.server.NIOServerCnxn; +import org.apache.zookeeper.server.NIOServerCnxnFactory; import org.apache.zookeeper.server.ZooKeeperServer; import org.apache.zookeeper.server.persistence.FileTxnSnapLog; @@ -32,7 +32,7 @@ public class EmbeddedZookeeper private final int port; private final File zkDataDir; private final ZooKeeperServer zkServer; - private final NIOServerCnxn.Factory cnxnFactory; + private final NIOServerCnxnFactory cnxnFactory; private final AtomicBoolean started = new AtomicBoolean(); private final AtomicBoolean stopped = new AtomicBoolean(); @@ -53,7 +53,8 @@ public EmbeddedZookeeper(int port) FileTxnSnapLog ftxn = new FileTxnSnapLog(zkDataDir, zkDataDir); zkServer.setTxnLogFactory(ftxn); - cnxnFactory = new NIOServerCnxn.Factory(new InetSocketAddress(this.port), 0); + cnxnFactory = new NIOServerCnxnFactory(); + cnxnFactory.configure(new InetSocketAddress(this.port), 0); } public void start() From 3bbabcefe030d9794fd329a4ec927f19cb75faca Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Tue, 13 Oct 2015 14:47:57 -0700 Subject: [PATCH 008/331] Read metastore info from zookeeper. --- presto-hive/pom.xml | 24 ++++ .../presto/hive/HiveClientModule.java | 4 +- .../hive/ZookeeperMetastoreMonitor.java | 125 ++++++++++++++++++ .../hive/ZookeeperServersetHiveCluster.java | 67 ++++++++++ .../ZookeeperServersetMetastoreConfig.java | 87 ++++++++++++ ...TestZookeeperServersetMetastoreConfig.java | 55 ++++++++ 6 files changed, 360 insertions(+), 2 deletions(-) create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperMetastoreMonitor.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetHiveCluster.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetMetastoreConfig.java create mode 100644 presto-hive/src/test/java/com/facebook/presto/hive/TestZookeeperServersetMetastoreConfig.java diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 94423d7afb8b..a8f459673f39 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -43,6 +43,24 @@ hive-apache + + org.apache.curator + curator-recipes + 2.8.0 + + + + org.apache.curator + curator-framework + 2.8.0 + + + + org.apache.curator + curator-client + 2.8.0 + + org.apache.thrift libthrift @@ -88,6 +106,12 @@ configuration + + com.googlecode.json-simple + json-simple + 1.1 + + com.google.guava guava 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 6a2e659ae27c..cc540c3076ea 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 @@ -90,8 +90,8 @@ public void configure(Binder binder) newExporter(binder).export(NamenodeStats.class).as(generatedNameOf(NamenodeStats.class)); binder.bind(HiveMetastoreClientFactory.class).in(Scopes.SINGLETON); - binder.bind(HiveCluster.class).to(StaticHiveCluster.class).in(Scopes.SINGLETON); - configBinder(binder).bindConfig(StaticMetastoreConfig.class); + binder.bind(HiveCluster.class).to(ZookeeperServersetHiveCluster.class).in(Scopes.SINGLETON); + configBinder(binder).bindConfig(ZookeeperServersetMetastoreConfig.class); binder.bind(TypeManager.class).toInstance(typeManager); binder.bind(PageIndexerFactory.class).toInstance(pageIndexerFactory); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperMetastoreMonitor.java b/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperMetastoreMonitor.java new file mode 100644 index 000000000000..002eefe5cc13 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperMetastoreMonitor.java @@ -0,0 +1,125 @@ +/* + * 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.net.HostAndPort; + +import io.airlift.log.Logger; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.framework.recipes.cache.PathChildrenCache; +import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; +import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; +import org.apache.curator.retry.ExponentialBackoffRetry; +import org.apache.curator.utils.ZKPaths; +import org.json.simple.JSONObject; +import org.json.simple.JSONValue; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.stream.Collectors; + +public class ZookeeperMetastoreMonitor implements PathChildrenCacheListener +{ + public static final Logger log = Logger.get(ZookeeperMetastoreMonitor.class); + private CuratorFramework client; + private PathChildrenCache cache; + private ConcurrentMap servers; // (Node_Name->HostAndPort) + + public ZookeeperMetastoreMonitor(String zkServer, String watchPath, int maxRetries, int retrySleepTime) + throws Exception + { + client = CuratorFrameworkFactory.newClient(zkServer, new ExponentialBackoffRetry(retrySleepTime, maxRetries)); + client.start(); + + cache = new PathChildrenCache(client, watchPath, true); // true indicating cache node contents in addition to the stat + try { + cache.start(); + } + catch (Exception ex) { + throw new RuntimeException("Curator PathCache Creation failed: " + ex.getMessage()); + } + + cache.getListenable().addListener(this); + servers = new ConcurrentHashMap<>(); + } + + public void close() + { + client.close(); + + try { + cache.close(); + } + catch (IOException ex) { + // do nothing + } + } + + public List getServers() + { + return servers.values().stream().collect(Collectors.toList()); + } + + private HostAndPort deserialize(byte[] bytes) + { + String serviceEndpoint = "serviceEndpoint"; + JSONObject data = (JSONObject) JSONValue.parse(new String(bytes)); + if (data != null && data.containsKey(serviceEndpoint)) { + Map hostPortMap = (Map) data.get(serviceEndpoint); + String host = hostPortMap.get("host").toString(); + int port = Integer.parseInt(hostPortMap.get("port").toString()); + return HostAndPort.fromParts(host, port); + } + else { + log.warn("failed to deserialize child node data"); + throw new IllegalArgumentException("No host:port found"); + } + } + + @Override + public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception + { + String node = ZKPaths.getNodeFromPath(event.getData().getPath()); + + switch (event.getType()) { + case CHILD_ADDED: { + HostAndPort hostPort = deserialize(event.getData().getData()); + log.info("child added: " + node + ": " + hostPort); + servers.put(node, hostPort); + break; + } + + case CHILD_UPDATED: { + HostAndPort hostPort = deserialize(event.getData().getData()); + log.info("child updated: " + node + ": " + hostPort); + servers.put(node, hostPort); + break; + } + + case CHILD_REMOVED: { + log.info("child removed: " + node); + servers.remove(node); + break; + } + + default: + log.info("connection state changed: " + node); + break; + } + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetHiveCluster.java b/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetHiveCluster.java new file mode 100644 index 000000000000..57b44d5bc05e --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetHiveCluster.java @@ -0,0 +1,67 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.hive; + +import com.facebook.presto.spi.PrestoException; +import com.google.common.net.HostAndPort; +import io.airlift.log.Logger; +import org.apache.thrift.transport.TTransportException; + +import javax.inject.Inject; + +import java.util.List; + +import static com.facebook.presto.hive.HiveErrorCode.HIVE_METASTORE_ERROR; +import static java.util.Objects.requireNonNull; + +/** + * Created by smittal on 10/9/15. + */ +public class ZookeeperServersetHiveCluster + implements HiveCluster +{ + private static final Logger log = Logger.get(ZookeeperServersetHiveCluster.class); + private final HiveMetastoreClientFactory clientFactory; + private ZookeeperMetastoreMonitor zkMetastoreMonitor; + + @Inject + public ZookeeperServersetHiveCluster(ZookeeperServersetMetastoreConfig config, HiveMetastoreClientFactory clientFactory) + throws Exception + { + String zkServerHostAndPort = requireNonNull(config.getZookeeperServerHostAndPort(), "zkServerHostAndPort is null"); + String zkMetastorePath = requireNonNull(config.getZookeeperMetastorePath(), "zkMetastorePath is null"); + int zkRetries = requireNonNull(config.getZookeeperMaxRetries(), "zkMaxRetried is null"); + int zkRetrySleepTime = requireNonNull(config.getZookeeperRetrySleepTime(), "zkRetrySleepTime is null"); + this.clientFactory = requireNonNull(clientFactory, "clientFactory is null"); + this.zkMetastoreMonitor = new ZookeeperMetastoreMonitor(zkServerHostAndPort, zkMetastorePath, zkRetries, zkRetrySleepTime); + } + + @Override + public HiveMetastoreClient createMetastoreClient() + { + List metastores = zkMetastoreMonitor.getServers(); + TTransportException lastException = null; + for (HostAndPort metastore : metastores) { + try { + log.info("Connecting to metastore at: " + metastore.toString()); + return clientFactory.create(metastore.getHostText(), metastore.getPort()); + } + catch (TTransportException e) { + lastException = e; + } + } + + throw new PrestoException(HIVE_METASTORE_ERROR, "Failed connecting to Hive metastore", lastException); + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetMetastoreConfig.java b/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetMetastoreConfig.java new file mode 100644 index 000000000000..339131fa3b9b --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetMetastoreConfig.java @@ -0,0 +1,87 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.hive; + +import io.airlift.configuration.Config; +import io.airlift.configuration.ConfigDescription; + +import javax.validation.constraints.Min; +import javax.validation.constraints.NotNull; + +/** + * Created by smittal on 10/9/15. + */ +public class ZookeeperServersetMetastoreConfig +{ + private String zookeeperServerHostAndPort; + private String zookeeperMetastorePath; + private int zookeeperRetrySleepTime = 500; // ms + private int zookeeperMaxRetries = 3; + + @NotNull + public String getZookeeperServerHostAndPort() + { + return zookeeperServerHostAndPort; + } + + @Config("hive.metastore.zookeeper.uri") + @ConfigDescription("Zookeeper Host and Port") + public ZookeeperServersetMetastoreConfig setZookeeperServerHostAndPort(String zookeeperServerHostAndPort) + { + this.zookeeperServerHostAndPort = zookeeperServerHostAndPort; + return this; + } + + @NotNull + public String getZookeeperMetastorePath() + { + return zookeeperMetastorePath; + } + + @Config("hive.metastore.zookeeper.path") + @ConfigDescription("Hive metastore Zookeeper path") + public ZookeeperServersetMetastoreConfig setZookeeperMetastorePath(String zkPath) + { + this.zookeeperMetastorePath = zkPath; + return this; + } + + @NotNull + public int getZookeeperRetrySleepTime() + { + return zookeeperRetrySleepTime; + } + + @Config("hive.metastore.zookeeper.retry.sleeptime") + @ConfigDescription("Zookeeper sleep time between reties") + public ZookeeperServersetMetastoreConfig setZookeeperRetrySleepTime(int zookeeperRetrySleepTime) + { + this.zookeeperRetrySleepTime = zookeeperRetrySleepTime; + return this; + } + + @Min(1) + public int getZookeeperMaxRetries() + { + return zookeeperMaxRetries; + } + + @Config("hive.metastore.zookeeper.max.retries") + @ConfigDescription("Zookeeper max reties") + public ZookeeperServersetMetastoreConfig setZookeeperMaxRetries(int zookeeperMaxRetries) + { + this.zookeeperMaxRetries = zookeeperMaxRetries; + return this; + } +} diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestZookeeperServersetMetastoreConfig.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestZookeeperServersetMetastoreConfig.java new file mode 100644 index 000000000000..bb1117697296 --- /dev/null +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestZookeeperServersetMetastoreConfig.java @@ -0,0 +1,55 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.hive; + +import com.google.common.collect.ImmutableMap; +import org.testng.annotations.Test; + +import java.util.Map; + +import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; +import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; +import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; + +public class TestZookeeperServersetMetastoreConfig +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(recordDefaults(ZookeeperServersetMetastoreConfig.class) + .setZookeeperMaxRetries(3) + .setZookeeperRetrySleepTime(500) + .setZookeeperMetastorePath(null) + .setZookeeperServerHostAndPort(null)); + } + + @Test + public void testExplicitPropertyMappingsSingleMetastore() + { + Map properties = new ImmutableMap.Builder() + .put("hive.metastore.zookeeper.uri", "localhost:2181") + .put("hive.metastore.zookeeper.path", "/zookeeper/path/") + .put("hive.metastore.zookeeper.retry.sleeptime", "200") + .put("hive.metastore.zookeeper.max.retries", "2") + .build(); + + ZookeeperServersetMetastoreConfig expected = new ZookeeperServersetMetastoreConfig() + .setZookeeperServerHostAndPort("localhost:2181") + .setZookeeperMetastorePath("/zookeeper/path/") + .setZookeeperRetrySleepTime(200) + .setZookeeperMaxRetries(2); + + assertFullMapping(properties, expected); + } +} From 202e620ec7f7743d0d0910573eef607c2eb2cd1f Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Tue, 13 Oct 2015 13:52:37 -0700 Subject: [PATCH 009/331] Add some tests and fix injection issue. --- presto-hive/pom.xml | 20 +++ .../presto/hive/HiveClientModule.java | 2 - .../presto/hive/HiveConnectorFactory.java | 8 + .../hive/MetastoreStaticClusterModule.java | 31 ++++ .../hive/MetastoreZkDiscoveryBasedModule.java | 31 ++++ .../ZookeeperServersetMetastoreConfig.java | 2 - .../hive/TestZookeeperMetastoreMonitor.java | 157 ++++++++++++++++++ .../facebook/presto/hive/util/TestUtils.java | 30 ++++ 8 files changed, 277 insertions(+), 4 deletions(-) create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/MetastoreStaticClusterModule.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/MetastoreZkDiscoveryBasedModule.java create mode 100644 presto-hive/src/test/java/com/facebook/presto/hive/TestZookeeperMetastoreMonitor.java create mode 100644 presto-hive/src/test/java/com/facebook/presto/hive/util/TestUtils.java diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index a8f459673f39..e75baa539500 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -61,6 +61,26 @@ 2.8.0 + + org.apache.curator + curator-test + 2.8.0 + test + + + + org.apache.zookeeper + zookeeper + 3.4.6 + test + + + + com.101tec + zkclient + test + + org.apache.thrift libthrift 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 cc540c3076ea..bc48cbfc7f0f 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 @@ -90,8 +90,6 @@ public void configure(Binder binder) newExporter(binder).export(NamenodeStats.class).as(generatedNameOf(NamenodeStats.class)); binder.bind(HiveMetastoreClientFactory.class).in(Scopes.SINGLETON); - binder.bind(HiveCluster.class).to(ZookeeperServersetHiveCluster.class).in(Scopes.SINGLETON); - configBinder(binder).bindConfig(ZookeeperServersetMetastoreConfig.class); binder.bind(TypeManager.class).toInstance(typeManager); binder.bind(PageIndexerFactory.class).toInstance(pageIndexerFactory); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveConnectorFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveConnectorFactory.java index 2725e938586e..922875f49317 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveConnectorFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveConnectorFactory.java @@ -93,6 +93,14 @@ public Connector create(String connectorId, Map config) new MBeanModule(), new JsonModule(), new HiveClientModule(connectorId, metastore, typeManager, pageIndexerFactory), + installModuleIf( + ZookeeperServersetMetastoreConfig.class, + zkMetastoreConfig -> zkMetastoreConfig.getZookeeperServerHostAndPort() == null, + new MetastoreStaticClusterModule()), + installModuleIf( + ZookeeperServersetMetastoreConfig.class, + zkMetastoreConfig -> zkMetastoreConfig.getZookeeperServerHostAndPort() != null, + new MetastoreZkDiscoveryBasedModule()), installModuleIf( SecurityConfig.class, security -> "none".equalsIgnoreCase(security.getSecuritySystem()), diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/MetastoreStaticClusterModule.java b/presto-hive/src/main/java/com/facebook/presto/hive/MetastoreStaticClusterModule.java new file mode 100644 index 000000000000..18bfa3d1eff1 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/MetastoreStaticClusterModule.java @@ -0,0 +1,31 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.hive; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Scopes; + +import static io.airlift.configuration.ConfigBinder.configBinder; + +public class MetastoreStaticClusterModule + implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bind(HiveCluster.class).to(StaticHiveCluster.class).in(Scopes.SINGLETON); + configBinder(binder).bindConfig(StaticMetastoreConfig.class); + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/MetastoreZkDiscoveryBasedModule.java b/presto-hive/src/main/java/com/facebook/presto/hive/MetastoreZkDiscoveryBasedModule.java new file mode 100644 index 000000000000..a4d84813ab76 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/MetastoreZkDiscoveryBasedModule.java @@ -0,0 +1,31 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.hive; + +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Scopes; + +import static io.airlift.configuration.ConfigBinder.configBinder; + +public class MetastoreZkDiscoveryBasedModule + implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bind(HiveCluster.class).to(ZookeeperServersetHiveCluster.class).in(Scopes.SINGLETON); + configBinder(binder).bindConfig(ZookeeperServersetMetastoreConfig.class); + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetMetastoreConfig.java b/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetMetastoreConfig.java index 339131fa3b9b..60867659d906 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetMetastoreConfig.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetMetastoreConfig.java @@ -29,7 +29,6 @@ public class ZookeeperServersetMetastoreConfig private int zookeeperRetrySleepTime = 500; // ms private int zookeeperMaxRetries = 3; - @NotNull public String getZookeeperServerHostAndPort() { return zookeeperServerHostAndPort; @@ -43,7 +42,6 @@ public ZookeeperServersetMetastoreConfig setZookeeperServerHostAndPort(String zo return this; } - @NotNull public String getZookeeperMetastorePath() { return zookeeperMetastorePath; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestZookeeperMetastoreMonitor.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestZookeeperMetastoreMonitor.java new file mode 100644 index 000000000000..5383e2e64198 --- /dev/null +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestZookeeperMetastoreMonitor.java @@ -0,0 +1,157 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.hive; + +import com.facebook.presto.hadoop.shaded.com.google.common.collect.ImmutableList; +import com.facebook.presto.hive.util.TestUtils; +import com.google.common.net.HostAndPort; +import io.airlift.log.Logger; +import org.I0Itec.zkclient.ZkClient; +import org.I0Itec.zkclient.exception.ZkMarshallingError; +import org.I0Itec.zkclient.serialize.ZkSerializer; +import org.apache.curator.test.TestingServer; +import org.json.simple.JSONObject; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeTest; +import org.testng.annotations.Test; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import static org.testng.Assert.assertTrue; + +public class TestZookeeperMetastoreMonitor +{ + private static final Logger log = Logger.get(TestZookeeperMetastoreMonitor.class); + + private ZookeeperMetastoreMonitor zkMetastoreMonitor; + private TestingServer zkServer; + private ZkClient zkClient; + private final String zkPath = "/metastores"; + + public TestZookeeperMetastoreMonitor() + throws Exception + { + zkServer = new TestingServer(TestUtils.findUnusedPort()); + zkClient = new ZkClient(zkServer.getConnectString(), 30_000, 30_000); + + // Set the serializer + zkClient.setZkSerializer(new ZkSerializer() { + @Override + public byte[] serialize(Object o) throws ZkMarshallingError + { + try { + return o.toString().getBytes("UTF-8"); + } + catch (Exception e) { + log.warn("Exception in serializing " + e); + } + return "".getBytes(); + } + + @Override + public Object deserialize(byte[] bytes) throws ZkMarshallingError + { + return null; + } + }); + } + + @AfterClass + public void destroy() + throws IOException + { + zkMetastoreMonitor.close(); + zkClient.close(); + zkServer.close(); + + } + + @BeforeTest + public void setUp() + throws Exception + { + log.info("Cleaning up zookeeper"); + zkClient.getChildren("/").stream() + .filter(child -> !child.equals("zookeeper")) + .forEach(child -> zkClient.deleteRecursive("/" + child)); + + zkClient.unsubscribeAll(); + + zkClient.createPersistent(zkPath); + zkMetastoreMonitor = new ZookeeperMetastoreMonitor(zkServer.getConnectString(), zkPath, 3, 500); + } + + @Test + public void testGetServers() throws Exception + { + List servers; + List expected; + assertTrue(zkMetastoreMonitor.getServers().isEmpty()); + + addServerToZk("nameNode1", "host1", 10001); + // Sleep for some time so that event can be propagated. + TimeUnit.MILLISECONDS.sleep(100); + servers = zkMetastoreMonitor.getServers(); + expected = ImmutableList.of(HostAndPort.fromParts("host1", 10001)); + assertTrue(servers.containsAll(expected) && expected.containsAll(servers)); + + addServerToZk("nameNode2", "host2", 10002); + // Sleep for some time so that event can be propagated. + TimeUnit.MILLISECONDS.sleep(100); + servers = zkMetastoreMonitor.getServers(); + expected = ImmutableList.of(HostAndPort.fromParts("host1", 10001), HostAndPort.fromParts("host2", 10002)); + assertTrue(servers.containsAll(expected) && expected.containsAll(servers)); + + // Change value of an existing name node + addServerToZk("nameNode2", "host2", 10003); + // Sleep for some time so that event can be propagated. + TimeUnit.MILLISECONDS.sleep(100); + servers = zkMetastoreMonitor.getServers(); + expected = ImmutableList.of(HostAndPort.fromParts("host1", 10001), HostAndPort.fromParts("host2", 10003)); + assertTrue(servers.containsAll(expected) && expected.containsAll(servers)); + + // Delete an existing name node + zkClient.delete(getPathForNameNode("nameNode1")); + // Sleep for some time so that event can be propagated. + TimeUnit.MILLISECONDS.sleep(100); + servers = zkMetastoreMonitor.getServers(); + expected = ImmutableList.of(HostAndPort.fromParts("host2", 10003)); + assertTrue(servers.containsAll(expected) && expected.containsAll(servers), servers.toString()); + } + + private void addServerToZk(String nameNode, String host, int port) + { + JSONObject serviceEndpoint = new JSONObject(); + serviceEndpoint.put("host", host); + serviceEndpoint.put("port", port); + JSONObject jsonObject = new JSONObject(); + jsonObject.put("serviceEndpoint", serviceEndpoint); + + String path = getPathForNameNode(nameNode); + + if (!zkClient.exists(path)) { + zkClient.createPersistent(path, jsonObject.toJSONString()); + } + else { + zkClient.writeData(path, jsonObject.toJSONString()); + } + } + + private String getPathForNameNode(String nameNode) + { + return zkPath + "/" + nameNode; + } +} diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/util/TestUtils.java b/presto-hive/src/test/java/com/facebook/presto/hive/util/TestUtils.java new file mode 100644 index 000000000000..4315bac667c2 --- /dev/null +++ b/presto-hive/src/test/java/com/facebook/presto/hive/util/TestUtils.java @@ -0,0 +1,30 @@ +/* + * 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.util; + +import java.io.IOException; +import java.net.ServerSocket; + +public final class TestUtils +{ + private TestUtils() {} + + public static int findUnusedPort() + throws IOException + { + try (ServerSocket socket = new ServerSocket(0)) { + return socket.getLocalPort(); + } + } +} From 93f55bbf1ebda92e6f4bf4929ba36ef8dadcd292 Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Wed, 14 Oct 2015 09:59:22 -0700 Subject: [PATCH 010/331] Remove Created by comments added by Idea. --- .../facebook/presto/hive/ZookeeperServersetHiveCluster.java | 3 --- .../presto/hive/ZookeeperServersetMetastoreConfig.java | 3 --- 2 files changed, 6 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetHiveCluster.java b/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetHiveCluster.java index 57b44d5bc05e..cbc434b74d2f 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetHiveCluster.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetHiveCluster.java @@ -25,9 +25,6 @@ import static com.facebook.presto.hive.HiveErrorCode.HIVE_METASTORE_ERROR; import static java.util.Objects.requireNonNull; -/** - * Created by smittal on 10/9/15. - */ public class ZookeeperServersetHiveCluster implements HiveCluster { diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetMetastoreConfig.java b/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetMetastoreConfig.java index 60867659d906..44c6f9d19188 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetMetastoreConfig.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetMetastoreConfig.java @@ -19,9 +19,6 @@ import javax.validation.constraints.Min; import javax.validation.constraints.NotNull; -/** - * Created by smittal on 10/9/15. - */ public class ZookeeperServersetMetastoreConfig { private String zookeeperServerHostAndPort; From 608333956fd263a0e3e103b1c52221c4d33f45fd Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Thu, 22 Oct 2015 13:42:25 -0700 Subject: [PATCH 011/331] Fix some warnings after merging latest master. --- .../facebook/presto/hive/ZookeeperServersetHiveCluster.java | 1 + .../facebook/presto/hive/TestZookeeperMetastoreMonitor.java | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetHiveCluster.java b/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetHiveCluster.java index cbc434b74d2f..48f869ec8ed0 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetHiveCluster.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetHiveCluster.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.hive; +import com.facebook.presto.hive.metastore.HiveMetastoreClient; import com.facebook.presto.spi.PrestoException; import com.google.common.net.HostAndPort; import io.airlift.log.Logger; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestZookeeperMetastoreMonitor.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestZookeeperMetastoreMonitor.java index 5383e2e64198..9ccf999edac4 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestZookeeperMetastoreMonitor.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestZookeeperMetastoreMonitor.java @@ -27,6 +27,7 @@ import org.testng.annotations.Test; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.List; import java.util.concurrent.TimeUnit; @@ -53,7 +54,7 @@ public TestZookeeperMetastoreMonitor() public byte[] serialize(Object o) throws ZkMarshallingError { try { - return o.toString().getBytes("UTF-8"); + return o.toString().getBytes(StandardCharsets.UTF_8); } catch (Exception e) { log.warn("Exception in serializing " + e); From 972d5d166bc8174c9cef50b1066f6efc9a07a02b Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Fri, 23 Oct 2015 11:39:02 -0700 Subject: [PATCH 012/331] Create ParquetHiveRecordCursor as user, so that hdfs reads happen as that user. --- .../parquet/ParquetRecordCursorProvider.java | 36 +++++++++++-------- 1 file changed, 22 insertions(+), 14 deletions(-) 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 eb6bb880bed3..3854449498da 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 @@ -27,10 +27,12 @@ import com.google.common.collect.ImmutableSet; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.security.UserGroupInformation; import org.joda.time.DateTimeZone; import javax.inject.Inject; +import java.security.PrivilegedExceptionAction; import java.util.List; import java.util.Optional; import java.util.Properties; @@ -88,20 +90,26 @@ public Optional createHiveRecordCursor( throw new IllegalArgumentException("Can not read Parquet column: " + unsupportedColumns); } - return Optional.of(new ParquetHiveRecordCursor( - configuration, - path, - start, - length, - schema, - partitionKeys, - columns, - useParquetColumnNames, - hiveStorageTimeZone, - typeManager, - isParquetPredicatePushdownEnabled(session), - effectivePredicate - )); + UserGroupInformation ugi = UserGroupInformation.createRemoteUser(session.getUser()); + try { + return ugi.doAs((PrivilegedExceptionAction>) () -> Optional.of(new ParquetHiveRecordCursor( + configuration, + path, + start, + length, + schema, + partitionKeys, + columns, + useParquetColumnNames, + hiveStorageTimeZone, + typeManager, + isParquetPredicatePushdownEnabled(session), + effectivePredicate + ))); + } + catch (Exception e) { + throw new RuntimeException(e); + } } private static Predicate isParquetSupportedType() From d01400182645e82f733760a2d938273c49dd5a88 Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Mon, 26 Oct 2015 13:20:43 -0700 Subject: [PATCH 013/331] Propagate user session information throughout hive metastore. --- .../facebook/presto/hive/HiveMetadata.java | 85 ++++++++++--------- .../facebook/presto/hive/HivePageSink.java | 11 ++- .../presto/hive/HivePageSinkProvider.java | 7 +- .../presto/hive/HivePartitionManager.java | 12 +-- .../presto/hive/HiveSplitManager.java | 7 +- .../facebook/presto/hive/HiveWriteUtils.java | 9 +- .../hive/metastore/CachingHiveMetastore.java | 30 +++---- .../presto/hive/metastore/HiveMetastore.java | 34 ++++---- .../presto/hive/AbstractTestHiveClient.java | 8 +- .../presto/hive/AbstractTestHiveClientS3.java | 20 ++--- .../hive/metastore/InMemoryHiveMetastore.java | 32 +++---- .../metastore/TestCachingHiveMetastore.java | 59 ++++++------- 12 files changed, 164 insertions(+), 150 deletions(-) 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 c8e48aa7e21b..a3ad7180b9de 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 @@ -231,14 +231,14 @@ public HivePartitionManager getPartitionManager() @Override public List listSchemaNames(ConnectorSession session) { - return metastore.getAllDatabases(); + return metastore.getAllDatabases(session.getUser()); } @Override public HiveTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName) { requireNonNull(tableName, "tableName is null"); - if (!metastore.getTable(tableName.getSchemaName(), tableName.getTableName()).isPresent()) { + if (!metastore.getTable(session.getUser(), tableName.getSchemaName(), tableName.getTableName()).isPresent()) { return null; } return new HiveTableHandle(connectorId, tableName.getSchemaName(), tableName.getTableName()); @@ -249,12 +249,12 @@ public ConnectorTableMetadata getTableMetadata(ConnectorSession session, Connect { requireNonNull(tableHandle, "tableHandle is null"); SchemaTableName tableName = schemaTableName(tableHandle); - return getTableMetadata(tableName); + return getTableMetadata(session, tableName); } - private ConnectorTableMetadata getTableMetadata(SchemaTableName tableName) + private ConnectorTableMetadata getTableMetadata(ConnectorSession session, SchemaTableName tableName) { - Optional table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()); + Optional
table = metastore.getTable(session.getUser(), tableName.getSchemaName(), tableName.getTableName()); if (!table.isPresent() || table.get().getTableType().equals(TableType.VIRTUAL_VIEW.name())) { throw new TableNotFoundException(tableName); } @@ -294,7 +294,7 @@ public List listTables(ConnectorSession session, String schemaN { ImmutableList.Builder tableNames = ImmutableList.builder(); for (String schemaName : listSchemas(session, schemaNameOrNull)) { - for (String tableName : metastore.getAllTables(schemaName).orElse(emptyList())) { + for (String tableName : metastore.getAllTables(session.getUser(), schemaName).orElse(emptyList())) { tableNames.add(new SchemaTableName(schemaName, tableName)); } } @@ -313,7 +313,7 @@ private List listSchemas(ConnectorSession session, String schemaNameOrNu public ColumnHandle getSampleWeightColumnHandle(ConnectorSession session, ConnectorTableHandle tableHandle) { SchemaTableName tableName = schemaTableName(tableHandle); - Optional
table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()); + Optional
table = metastore.getTable(session.getUser(), tableName.getSchemaName(), tableName.getTableName()); if (!table.isPresent()) { throw new TableNotFoundException(tableName); } @@ -335,7 +335,7 @@ public boolean canCreateSampledTables(ConnectorSession session) public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) { SchemaTableName tableName = schemaTableName(tableHandle); - Optional
table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()); + Optional
table = metastore.getTable(session.getUser(), tableName.getSchemaName(), tableName.getTableName()); if (!table.isPresent()) { throw new TableNotFoundException(tableName); } @@ -356,7 +356,7 @@ public Map> listTableColumns(ConnectorSess ImmutableMap.Builder> columns = ImmutableMap.builder(); for (SchemaTableName tableName : listTables(session, prefix)) { try { - columns.put(tableName, getTableMetadata(tableName).getColumns()); + columns.put(tableName, getTableMetadata(session, tableName).getColumns()); } catch (HiveViewNotSupportedException e) { // view is not supported @@ -397,17 +397,18 @@ public void createTable(ConnectorSession session, ConnectorTableMetadata tableMe List partitionedBy = getPartitionedBy(tableMetadata.getProperties()); List columnHandles = getColumnHandles(connectorId, tableMetadata, ImmutableSet.copyOf(partitionedBy)); HiveStorageFormat hiveStorageFormat = getHiveStorageFormat(tableMetadata.getProperties()); - createTable(schemaName, tableName, tableMetadata.getOwner(), columnHandles, hiveStorageFormat, partitionedBy); + createTable(session, schemaName, tableName, tableMetadata.getOwner(), columnHandles, hiveStorageFormat, partitionedBy); } - public void createTable(String schemaName, + public void createTable(ConnectorSession session, + String schemaName, String tableName, String tableOwner, List columnHandles, HiveStorageFormat hiveStorageFormat, List partitionedBy) { - Path targetPath = getTableDefaultLocation(metastore, hdfsEnvironment, schemaName, tableName); + Path targetPath = getTableDefaultLocation(session, metastore, hdfsEnvironment, schemaName, tableName); // verify the target directory for the table if (pathExists(hdfsEnvironment, targetPath)) { @@ -415,10 +416,11 @@ public void createTable(String schemaName, } createDirectory(hdfsEnvironment, targetPath); - createTable(schemaName, tableName, tableOwner, columnHandles, hiveStorageFormat, partitionedBy, targetPath); + createTable(session, schemaName, tableName, tableOwner, columnHandles, hiveStorageFormat, partitionedBy, targetPath); } - private Table createTable(String schemaName, + private Table createTable(ConnectorSession session, + String schemaName, String tableName, String tableOwner, List columnHandles, @@ -484,7 +486,7 @@ else if (!partitionColumnNames.contains(name)) { ImmutableMap.of(), ImmutableMap.of())); - metastore.createTable(table); + metastore.createTable(session.getUser(), table); return table; } @@ -496,7 +498,7 @@ public void addColumn(ConnectorSession session, ConnectorTableHandle tableHandle } HiveTableHandle handle = checkType(tableHandle, HiveTableHandle.class, "tableHandle"); - Optional
tableMetadata = metastore.getTable(handle.getSchemaName(), handle.getTableName()); + Optional
tableMetadata = metastore.getTable(session.getUser(), handle.getSchemaName(), handle.getTableName()); if (!tableMetadata.isPresent()) { throw new TableNotFoundException(handle.getSchemaTableName()); } @@ -509,7 +511,7 @@ public void addColumn(ConnectorSession session, ConnectorTableHandle tableHandle sd.setCols(columns.build()); table.setSd(sd); - metastore.alterTable(handle.getSchemaName(), handle.getTableName(), table); + metastore.alterTable(session.getUser(), handle.getSchemaName(), handle.getTableName(), table); } @Override @@ -521,7 +523,7 @@ public void renameColumn(ConnectorSession session, ConnectorTableHandle tableHan HiveTableHandle hiveTableHandle = checkType(tableHandle, HiveTableHandle.class, "tableHandle"); HiveColumnHandle sourceHandle = checkType(source, HiveColumnHandle.class, "columnHandle"); - Optional
tableMetadata = metastore.getTable(hiveTableHandle.getSchemaName(), hiveTableHandle.getTableName()); + Optional
tableMetadata = metastore.getTable(session.getUser(), hiveTableHandle.getSchemaName(), hiveTableHandle.getTableName()); if (!tableMetadata.isPresent()) { throw new TableNotFoundException(hiveTableHandle.getSchemaTableName()); } @@ -538,7 +540,7 @@ public void renameColumn(ConnectorSession session, ConnectorTableHandle tableHan } sd.setCols(columns.build()); table.setSd(sd); - metastore.alterTable(hiveTableHandle.getSchemaName(), hiveTableHandle.getTableName(), table); + metastore.alterTable(session.getUser(), hiveTableHandle.getSchemaName(), hiveTableHandle.getTableName(), table); } @Override @@ -550,14 +552,14 @@ public void renameTable(ConnectorSession session, ConnectorTableHandle tableHand HiveTableHandle handle = checkType(tableHandle, HiveTableHandle.class, "tableHandle"); SchemaTableName tableName = schemaTableName(tableHandle); - Optional
source = metastore.getTable(handle.getSchemaName(), handle.getTableName()); + Optional
source = metastore.getTable(session.getUser(), handle.getSchemaName(), handle.getTableName()); if (!source.isPresent()) { throw new TableNotFoundException(tableName); } Table table = source.get(); table.setDbName(newTableName.getSchemaName()); table.setTableName(newTableName.getTableName()); - metastore.alterTable(handle.getSchemaName(), handle.getTableName(), table); + metastore.alterTable(session.getUser(), handle.getSchemaName(), handle.getTableName(), table); } @Override @@ -570,7 +572,7 @@ public void dropTable(ConnectorSession session, ConnectorTableHandle tableHandle throw new PrestoException(PERMISSION_DENIED, "DROP TABLE is disabled in this Hive catalog"); } - Optional
target = metastore.getTable(handle.getSchemaName(), handle.getTableName()); + Optional
target = metastore.getTable(session.getUser(), handle.getSchemaName(), handle.getTableName()); if (!target.isPresent()) { throw new TableNotFoundException(tableName); } @@ -579,7 +581,7 @@ public void dropTable(ConnectorSession session, ConnectorTableHandle tableHandle if (!session.getUser().equals(table.getOwner())) { throw new PrestoException(PERMISSION_DENIED, format("Unable to drop table '%s': owner of the table is different from session user", table)); } - metastore.dropTable(handle.getSchemaName(), handle.getTableName()); + metastore.dropTable(session.getUser(), handle.getSchemaName(), handle.getTableName()); } @Override @@ -599,7 +601,7 @@ public HiveOutputTableHandle beginCreateTable(ConnectorSession session, Connecto List columnHandles = getColumnHandles(connectorId, tableMetadata, ImmutableSet.copyOf(partitionedBy)); - Path targetPath = getTableDefaultLocation(metastore, hdfsEnvironment, schemaName, tableName); + Path targetPath = getTableDefaultLocation(session, metastore, hdfsEnvironment, schemaName, tableName); // verify the target directory for the table if (pathExists(hdfsEnvironment, targetPath)) { @@ -633,7 +635,7 @@ public void commitCreateTable(ConnectorSession session, ConnectorOutputTableHand .map(partitionUpdateCodec::fromJson) .collect(toList()); - Path targetPath = getTableDefaultLocation(metastore, hdfsEnvironment, handle.getSchemaName(), handle.getTableName()); + Path targetPath = getTableDefaultLocation(session, metastore, hdfsEnvironment, handle.getSchemaName(), handle.getTableName()); Path writePath = new Path(handle.getWritePath().get()); // rename if using a temporary directory @@ -649,11 +651,12 @@ public void commitCreateTable(ConnectorSession session, ConnectorOutputTableHand renameDirectory(hdfsEnvironment, handle.getSchemaName(), handle.getTableName(), writePath, targetPath); } - PartitionCommitter partitionCommitter = new PartitionCommitter(handle.getSchemaName(), handle.getTableName(), metastore, PARTITION_COMMIT_BATCH_SIZE); + PartitionCommitter partitionCommitter = new PartitionCommitter(session, handle.getSchemaName(), handle.getTableName(), metastore, PARTITION_COMMIT_BATCH_SIZE); try { partitionUpdates = PartitionUpdate.mergePartitionUpdates(partitionUpdates); Table table = createTable( + session, handle.getSchemaName(), handle.getTableName(), handle.getTableOwner(), @@ -691,7 +694,7 @@ public HiveInsertTableHandle beginInsert(ConnectorSession session, ConnectorTabl verifyJvmTimeZone(); SchemaTableName tableName = schemaTableName(tableHandle); - Optional
table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()); + Optional
table = metastore.getTable(session.getUser(), tableName.getSchemaName(), tableName.getTableName()); if (!table.isPresent()) { throw new TableNotFoundException(tableName); } @@ -724,6 +727,7 @@ public HiveInsertTableHandle beginInsert(ConnectorSession session, ConnectorTabl private static class PartitionCommitter implements Closeable { + private final ConnectorSession session; private final String schemaName; private final String tableName; private final HiveMetastore metastore; @@ -731,8 +735,9 @@ private static class PartitionCommitter private final List batch; private final List createdPartitions = new ArrayList<>(); - public PartitionCommitter(String schemaName, String tableName, HiveMetastore metastore, int batchSize) + public PartitionCommitter(ConnectorSession session, String schemaName, String tableName, HiveMetastore metastore, int batchSize) { + this.session = session; this.schemaName = schemaName; this.tableName = tableName; this.metastore = metastore; @@ -766,7 +771,7 @@ public void abort() // drop created partitions for (Partition createdPartition : getCreatedPartitions()) { try { - metastore.dropPartition(schemaName, tableName, createdPartition.getValues()); + metastore.dropPartition(session.getUser(), schemaName, tableName, createdPartition.getValues()); } catch (Exception e) { log.error(e, "Error rolling back new partition '%s' in table '%s.%s", createdPartition.getValues(), schemaName, tableName); @@ -776,7 +781,7 @@ public void abort() private void addBatch() { - metastore.addPartitions(schemaName, tableName, batch); + metastore.addPartitions(session.getUser(), schemaName, tableName, batch); createdPartitions.addAll(batch); batch.clear(); } @@ -793,11 +798,11 @@ public void commitInsert(ConnectorSession session, ConnectorInsertTableHandle in .collect(toList()); HiveStorageFormat storageFormat = handle.getHiveStorageFormat(); - PartitionCommitter partitionCommitter = new PartitionCommitter(handle.getSchemaName(), handle.getTableName(), metastore, PARTITION_COMMIT_BATCH_SIZE); + PartitionCommitter partitionCommitter = new PartitionCommitter(session, handle.getSchemaName(), handle.getTableName(), metastore, PARTITION_COMMIT_BATCH_SIZE); try { partitionUpdates = PartitionUpdate.mergePartitionUpdates(partitionUpdates); - Optional
table = metastore.getTable(handle.getSchemaName(), handle.getTableName()); + Optional
table = metastore.getTable(session.getUser(), handle.getSchemaName(), handle.getTableName()); if (!table.isPresent()) { throw new TableNotFoundException(new SchemaTableName(handle.getSchemaName(), handle.getTableName())); } @@ -890,7 +895,7 @@ public void rollbackInsert(ConnectorSession session, ConnectorInsertTableHandle } // Otherwise, insert was directly into the target table and partitions, and all must be checked for temp files - Optional
table = metastore.getTable(handle.getSchemaName(), handle.getTableName()); + Optional
table = metastore.getTable(session.getUser(), handle.getSchemaName(), handle.getTableName()); if (!table.isPresent()) { log.error("Error rolling back insert into table %s.%s. Table was dropped during insert, and data directory may contain temporary data", handle.getSchemaName(), handle.getTableName()); return; @@ -904,10 +909,10 @@ public void rollbackInsert(ConnectorSession session, ConnectorInsertTableHandle // check every existing partition that is outside for the base directory if (!table.get().getPartitionKeys().isEmpty()) { - List partitionNames = metastore.getPartitionNames(handle.getSchemaName(), handle.getTableName()) + List partitionNames = metastore.getPartitionNames(session.getUser(), handle.getSchemaName(), handle.getTableName()) .orElse(ImmutableList.of()); for (List partitionNameBatch : Iterables.partition(partitionNames, 10)) { - metastore.getPartitionsByNames(handle.getSchemaName(), handle.getTableName(), partitionNameBatch).orElse(ImmutableMap.of()).values().stream() + metastore.getPartitionsByNames(session.getUser(), handle.getSchemaName(), handle.getTableName(), partitionNameBatch).orElse(ImmutableMap.of()).values().stream() .map(partition -> partition.getSd().getLocation()) .filter(location -> !location.startsWith(tableDirectory)) .forEach(locationsToClean::add); @@ -1135,7 +1140,7 @@ public void createView(ConnectorSession session, SchemaTableName viewName, Strin ImmutableMap.of())); try { - metastore.createTable(table); + metastore.createTable(session.getUser(), table); } catch (TableAlreadyExistsException e) { throw new ViewAlreadyExistsException(e.getTableName()); @@ -1151,7 +1156,7 @@ public void dropView(ConnectorSession session, SchemaTableName viewName) } try { - metastore.dropTable(viewName.getSchemaName(), viewName.getTableName()); + metastore.dropTable(session.getUser(), viewName.getSchemaName(), viewName.getTableName()); } catch (TableNotFoundException e) { throw new ViewNotFoundException(e.getTableName()); @@ -1163,7 +1168,7 @@ public List listViews(ConnectorSession session, String schemaNa { ImmutableList.Builder tableNames = ImmutableList.builder(); for (String schemaName : listSchemas(session, schemaNameOrNull)) { - for (String tableName : metastore.getAllViews(schemaName).orElse(emptyList())) { + for (String tableName : metastore.getAllViews(session.getUser(), schemaName).orElse(emptyList())) { tableNames.add(new SchemaTableName(schemaName, tableName)); } } @@ -1183,7 +1188,7 @@ public Map getViews(ConnectorSession s } for (SchemaTableName schemaTableName : tableNames) { - Optional
table = metastore.getTable(schemaTableName.getSchemaName(), schemaTableName.getTableName()); + Optional
table = metastore.getTable(session.getUser(), schemaTableName.getSchemaName(), schemaTableName.getTableName()); if (table.isPresent() && HiveUtil.isPrestoView(table.get())) { views.put(schemaTableName, new ConnectorViewDefinition( schemaTableName, @@ -1215,7 +1220,7 @@ public OptionalLong metadataDelete(ConnectorSession session, ConnectorTableHandl //for (HivePartition hivePartition : layoutHandle.getOrComputePartitions(this, session, tableHandle)) { for (HivePartition hivePartition : getOrComputePartitions(layoutHandle, session, tableHandle)) { - metastore.dropPartitionByName(handle.getSchemaName(), handle.getTableName(), hivePartition.getPartitionId()); + metastore.dropPartitionByName(session.getUser(), handle.getSchemaName(), handle.getTableName(), hivePartition.getPartitionId()); } // it is too expensive to determine the exact number of deleted rows return OptionalLong.empty(); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSink.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSink.java index 4e991d1547c4..6444e05878f1 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSink.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSink.java @@ -15,6 +15,7 @@ import com.facebook.presto.hive.metastore.HiveMetastore; import com.facebook.presto.spi.ConnectorPageSink; +import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.Page; import com.facebook.presto.spi.PageIndexer; import com.facebook.presto.spi.PageIndexerFactory; @@ -88,6 +89,8 @@ public class HivePageSink implements ConnectorPageSink { + private final ConnectorSession session; + private final String schemaName; private final String tableName; @@ -122,6 +125,7 @@ public class HivePageSink private HiveRecordWriter[] writers = new HiveRecordWriter[0]; public HivePageSink( + ConnectorSession session, String schemaName, String tableName, boolean isCreateTable, @@ -138,6 +142,7 @@ public HivePageSink( boolean immutablePartitions, JsonCodec partitionUpdateCodec) { + this.session = requireNonNull(session, "session is null"); this.schemaName = requireNonNull(schemaName, "schemaName is null"); this.tableName = requireNonNull(tableName, "tableName is null"); @@ -210,7 +215,7 @@ public HivePageSink( conf = new JobConf(hdfsEnvironment.getConfiguration(writePath.get())); } else { - Optional
table = metastore.getTable(schemaName, tableName); + Optional
table = metastore.getTable(session.getUser(), schemaName, tableName); if (!table.isPresent()) { throw new PrestoException(HIVE_INVALID_METADATA, format("Table %s.%s was dropped during insert", schemaName, tableName)); } @@ -287,7 +292,7 @@ private HiveRecordWriter createWriter(List partitionRow) // attempt to get the existing partition (if this is an existing partitioned table) Optional partition = Optional.empty(); if (!partitionRow.isEmpty() && table != null) { - partition = metastore.getPartition(schemaName, tableName, partitionName); + partition = metastore.getPartition(session.getUser(), schemaName, tableName, partitionName); } if (!partition.isPresent()) { @@ -303,7 +308,7 @@ private HiveRecordWriter createWriter(List partitionRow) .map(HiveType::toHiveType) .map(HiveType::getHiveTypeName) .collect(Collectors.joining(":"))); - target = getTableDefaultLocation(metastore, hdfsEnvironment, schemaName, tableName).toString(); + target = getTableDefaultLocation(session, metastore, hdfsEnvironment, schemaName, tableName).toString(); if (!partitionRow.isEmpty()) { // verify the target directory for the partition does not already exist diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSinkProvider.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSinkProvider.java index 7cb88075d5f6..313fe5f5592d 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSinkProvider.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSinkProvider.java @@ -64,19 +64,20 @@ public HivePageSinkProvider( public ConnectorPageSink createPageSink(ConnectorSession session, ConnectorOutputTableHandle tableHandle) { HiveWritableTableHandle handle = checkType(tableHandle, HiveOutputTableHandle.class, "tableHandle"); - return createPageSink(handle, true); + return createPageSink(session, handle, true); } @Override public ConnectorPageSink createPageSink(ConnectorSession session, ConnectorInsertTableHandle tableHandle) { HiveInsertTableHandle handle = checkType(tableHandle, HiveInsertTableHandle.class, "tableHandle"); - return createPageSink(handle, false); + return createPageSink(session, handle, false); } - private ConnectorPageSink createPageSink(HiveWritableTableHandle handle, boolean isCreateTable) + private ConnectorPageSink createPageSink(ConnectorSession session, HiveWritableTableHandle handle, boolean isCreateTable) { return new HivePageSink( + session, handle.getSchemaName(), handle.getTableName(), isCreateTable, diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java index 37eb15d30025..b41f6cb702fc 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java @@ -100,7 +100,7 @@ public HivePartitionResult getPartitions(ConnectorSession session, HiveMetastore } SchemaTableName tableName = hiveTableHandle.getSchemaTableName(); - Table table = getTable(metastore, tableName); + Table table = getTable(session, metastore, tableName); Optional bucket = getHiveBucket(table, effectivePredicate.extractFixedValues()); TupleDomain compactEffectivePredicate = toCompactTupleDomain(effectivePredicate, domainCompactionThreshold); @@ -110,7 +110,7 @@ public HivePartitionResult getPartitions(ConnectorSession session, HiveMetastore } List partitionColumns = getPartitionKeyColumnHandles(connectorId, table); - List partitionNames = getFilteredPartitionNames(metastore, tableName, partitionColumns, effectivePredicate); + List partitionNames = getFilteredPartitionNames(session, metastore, tableName, partitionColumns, effectivePredicate); // do a final pass to filter based on fields that could not be used to filter the partitions ImmutableList.Builder partitions = ImmutableList.builder(); @@ -164,9 +164,9 @@ private Optional> parseValuesAndFilte return Optional.of(builder.build()); } - private Table getTable(HiveMetastore metastore, SchemaTableName tableName) + private Table getTable(ConnectorSession session, HiveMetastore metastore, SchemaTableName tableName) { - Optional
target = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()); + Optional
target = metastore.getTable(session.getUser(), tableName.getSchemaName(), tableName.getTableName()); if (!target.isPresent()) { throw new TableNotFoundException(tableName); } @@ -185,7 +185,7 @@ private Table getTable(HiveMetastore metastore, SchemaTableName tableName) return table; } - private List getFilteredPartitionNames(HiveMetastore metastore, SchemaTableName tableName, List partitionKeys, TupleDomain effectivePredicate) + private List getFilteredPartitionNames(ConnectorSession session, HiveMetastore metastore, SchemaTableName tableName, List partitionKeys, TupleDomain effectivePredicate) { List filter = new ArrayList<>(); for (HiveColumnHandle partitionKey : partitionKeys) { @@ -217,7 +217,7 @@ else if ((value instanceof Boolean) || (value instanceof Double) || (value insta } // fetch the partition names - return metastore.getPartitionNamesByParts(tableName.getSchemaName(), tableName.getTableName(), filter) + return metastore.getPartitionNamesByParts(session.getUser(), tableName.getSchemaName(), tableName.getTableName(), filter) .orElseThrow(() -> new TableNotFoundException(tableName)); } 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 8f63692545bc..1506fc8f3c64 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 @@ -157,11 +157,11 @@ public ConnectorSplitSource getSplits(ConnectorSession session, ConnectorTableLa // sort partitions partitions = Ordering.natural().onResultOf(HivePartition::getPartitionId).reverse().sortedCopy(partitions); - Optional
table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()); + Optional
table = metastore.getTable(session.getUser(), tableName.getSchemaName(), tableName.getTableName()); if (!table.isPresent()) { throw new TableNotFoundException(tableName); } - Iterable hivePartitions = getPartitionMetadata(table.get(), tableName, partitions); + Iterable hivePartitions = getPartitionMetadata(session, table.get(), tableName, partitions); HiveSplitLoader hiveSplitLoader = new BackgroundHiveSplitLoader( connectorId, @@ -185,7 +185,7 @@ public ConnectorSplitSource getSplits(ConnectorSession session, ConnectorTableLa return splitSource; } - private Iterable getPartitionMetadata(Table table, SchemaTableName tableName, List hivePartitions) + private Iterable getPartitionMetadata(ConnectorSession session, Table table, SchemaTableName tableName, List hivePartitions) { if (hivePartitions.isEmpty()) { return ImmutableList.of(); @@ -201,6 +201,7 @@ private Iterable getPartitionMetadata(Table table, Schema Iterable> partitionNameBatches = partitionExponentially(hivePartitions, minPartitionBatchSize, maxPartitionBatchSize); Iterable> partitionBatches = transform(partitionNameBatches, partitionBatch -> { Optional> batch = metastore.getPartitionsByNames( + session.getUser(), tableName.getSchemaName(), tableName.getTableName(), Lists.transform(partitionBatch, HivePartition::getPartitionId)); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java index 58e7aa397a98..0ebaa8158583 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java @@ -14,6 +14,7 @@ package com.facebook.presto.hive; import com.facebook.presto.hive.metastore.HiveMetastore; +import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.SchemaNotFoundException; import com.facebook.presto.spi.SchemaTableName; @@ -301,9 +302,9 @@ private static void checkWritable( } } - public static Path getTableDefaultLocation(HiveMetastore metastore, HdfsEnvironment hdfsEnvironment, String schemaName, String tableName) + public static Path getTableDefaultLocation(ConnectorSession session, HiveMetastore metastore, HdfsEnvironment hdfsEnvironment, String schemaName, String tableName) { - String location = getDatabase(metastore, schemaName).getLocationUri(); + String location = getDatabase(session, metastore, schemaName).getLocationUri(); if (isNullOrEmpty(location)) { throw new PrestoException(HIVE_DATABASE_LOCATION_ERROR, format("Database '%s' location is not set", schemaName)); } @@ -319,9 +320,9 @@ public static Path getTableDefaultLocation(HiveMetastore metastore, HdfsEnvironm return new Path(databasePath, tableName); } - private static Database getDatabase(HiveMetastore metastore, String database) + private static Database getDatabase(ConnectorSession session, HiveMetastore metastore, String database) { - return metastore.getDatabase(database).orElseThrow(() -> new SchemaNotFoundException(database)); + return metastore.getDatabase(session.getUser(), database).orElseThrow(() -> new SchemaNotFoundException(database)); } public static boolean pathExists(HdfsEnvironment hdfsEnvironment, Path path) 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 75d7074a543a..ed668aabb338 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 @@ -305,7 +305,7 @@ private static Map getAll(LoadingCache cache, Iterable key } @Override - public List getAllDatabases() + public List getAllDatabases(String user) { return get(databaseNamesCache, ""); } @@ -328,7 +328,7 @@ private List loadAllDatabases() } @Override - public Optional getDatabase(String databaseName) + public Optional getDatabase(String user, String databaseName) { return get(databaseCache, databaseName); } @@ -355,7 +355,7 @@ private Optional loadDatabase(String databaseName) } @Override - public Optional> getAllTables(String databaseName) + public Optional> getAllTables(String user, String databaseName) { return get(tableNamesCache, databaseName); } @@ -398,13 +398,13 @@ private Optional> loadAllTables(String databaseName) } @Override - public Optional
getTable(String databaseName, String tableName) + public Optional
getTable(String user, String databaseName, String tableName) { return get(tableCache, HiveTableName.table(databaseName, tableName)); } @Override - public Optional> getAllViews(String databaseName) + public Optional> getAllViews(String user, String databaseName) { return get(viewNamesCache, databaseName); } @@ -432,7 +432,7 @@ private Optional> loadAllViews(String databaseName) } @Override - public void createTable(Table table) + public void createTable(String user, Table table) { try { retry() @@ -467,7 +467,7 @@ public void createTable(Table table) } @Override - public void dropTable(String databaseName, String tableName) + public void dropTable(String user, String databaseName, String tableName) { try { retry() @@ -506,7 +506,7 @@ protected void invalidateTable(String databaseName, String tableName) } @Override - public void alterTable(String databaseName, String tableName, Table table) + public void alterTable(String user, String databaseName, String tableName, Table table) { try { retry() @@ -570,7 +570,7 @@ private Optional
loadTable(HiveTableName hiveTableName) } @Override - public Optional> getPartitionNames(String databaseName, String tableName) + public Optional> getPartitionNames(String user, String databaseName, String tableName) { return get(partitionNamesCache, HiveTableName.table(databaseName, tableName)); } @@ -602,7 +602,7 @@ private Optional> loadPartitionNames(HiveTableName hiveTableName) } @Override - public Optional> getPartitionNamesByParts(String databaseName, String tableName, List parts) + public Optional> getPartitionNamesByParts(String user, String databaseName, String tableName, List parts) { return get(partitionFilterCache, PartitionFilter.partitionFilter(databaseName, tableName, parts)); } @@ -632,7 +632,7 @@ private Optional> loadPartitionNamesByParts(PartitionFilter partiti } @Override - public void addPartitions(String databaseName, String tableName, List partitions) + public void addPartitions(String user, String databaseName, String tableName, List partitions) { if (partitions.isEmpty()) { return; @@ -676,7 +676,7 @@ public void addPartitions(String databaseName, String tableName, List } @Override - public void dropPartition(String databaseName, String tableName, List parts) + public void dropPartition(String user, String databaseName, String tableName, List parts) { try { retry() @@ -707,7 +707,7 @@ public void dropPartition(String databaseName, String tableName, List pa } @Override - public void dropPartitionByName(String databaseName, String tableName, String partitionName) + public void dropPartitionByName(String user, String databaseName, String tableName, String partitionName) { try { retry() @@ -753,7 +753,7 @@ private void invalidatePartitionCache(String databaseName, String tableName) } @Override - public Optional> getPartitionsByNames(String databaseName, String tableName, List partitionNames) + public Optional> getPartitionsByNames(String user, String databaseName, String tableName, List partitionNames) { Iterable names = transform(partitionNames, name -> HivePartitionName.partition(databaseName, tableName, name)); @@ -769,7 +769,7 @@ public Optional> getPartitionsByNames(String databaseName } @Override - public Optional getPartition(String databaseName, String tableName, String partitionName) + public Optional getPartition(String user, String databaseName, String tableName, String partitionName) { HivePartitionName name = HivePartitionName.partition(databaseName, tableName, partitionName); return get(partitionCache, name); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/HiveMetastore.java b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/HiveMetastore.java index 49e9b71da794..066cb9576a23 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/HiveMetastore.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/HiveMetastore.java @@ -30,43 +30,43 @@ public interface HiveMetastore { String DEFAULT_DATABASE_NAME = "default"; - void createTable(Table table); + void createTable(String user, Table table); - void dropTable(String databaseName, String tableName); + void dropTable(String user, String databaseName, String tableName); - void alterTable(String databaseName, String tableName, Table table); + void alterTable(String user, String databaseName, String tableName, Table table); @Managed void flushCache(); - List getAllDatabases(); + List getAllDatabases(String user); - Optional> getAllTables(String databaseName); + Optional> getAllTables(String user, String databaseName); - Optional> getAllViews(String databaseName); + Optional> getAllViews(String user, String databaseName); - Optional getDatabase(String databaseName); + Optional getDatabase(String user, String databaseName); /** * Adds partitions to the table in a single atomic task. The implementation * must either add all partitions and return normally, or add no partitions and * throw an exception. */ - void addPartitions(String databaseName, String tableName, List partitions); + void addPartitions(String user, String databaseName, String tableName, List partitions); - void dropPartition(String databaseName, String tableName, List parts); + void dropPartition(String user, String databaseName, String tableName, List parts); - void dropPartitionByName(String databaseName, String tableName, String partitionName); + void dropPartitionByName(String user, String databaseName, String tableName, String partitionName); - Optional> getPartitionNames(String databaseName, String tableName); + Optional> getPartitionNames(String user, String databaseName, String tableName); - Optional> getPartitionNamesByParts(String databaseName, String tableName, List parts); + Optional> getPartitionNamesByParts(String user, String databaseName, String tableName, List parts); - Optional getPartition(String databaseName, String tableName, String partitionName); + Optional getPartition(String user, String databaseName, String tableName, String partitionName); - Optional> getPartitionsByNames(String databaseName, String tableName, List partitionNames); + Optional> getPartitionsByNames(String user, String databaseName, String tableName, List partitionNames); - Optional
getTable(String databaseName, String tableName); + Optional
getTable(String user, String databaseName, String tableName); Set getRoles(String user); @@ -81,7 +81,7 @@ default boolean isDatabaseOwner(String user, String databaseName) return true; } - Optional databaseMetadata = getDatabase(databaseName); + Optional databaseMetadata = getDatabase(user, databaseName); if (!databaseMetadata.isPresent()) { return false; } @@ -101,7 +101,7 @@ default boolean isDatabaseOwner(String user, String databaseName) default boolean isTableOwner(String user, String databaseName, String tableName) { // a table can only be owned by a user - Optional
table = getTable(databaseName, tableName); + Optional
table = getTable(user, databaseName, tableName); return table.isPresent() && user.equals(table.get().getOwner()); } } 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 1cd4adf8046d..8a5692904b4c 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 @@ -1564,7 +1564,7 @@ protected void doCreateEmptyTable(SchemaTableName tableName, HiveStorageFormat s assertEquals(tableMetadata.getColumns(), createTableColumns); // verify table format - Table table = getMetastoreClient(tableName.getSchemaName()).getTable(tableName.getSchemaName(), tableName.getTableName()).get(); + Table table = getMetastoreClient(tableName.getSchemaName()).getTable(SESSION.getUser(), tableName.getSchemaName(), tableName.getTableName()).get(); if (!table.getSd().getInputFormat().equals(storageFormat.getInputFormat())) { assertEquals(table.getSd().getInputFormat(), storageFormat.getInputFormat()); } @@ -1678,7 +1678,7 @@ protected Set listAllDataFiles(ConnectorInsertTableHandle tableHandle) protected Set listAllDataFiles(String schemaName, String tableName) throws IOException { - Table table = metastoreClient.getTable(schemaName, tableName).get(); + Table table = metastoreClient.getTable(SESSION.getUser(), schemaName, tableName).get(); Path path = new Path(table.getSd().getLocation()); Set existingFiles = new HashSet<>(); return listAllDataFiles(path, existingFiles); @@ -1715,7 +1715,7 @@ private void doInsertPartitioned(HiveStorageFormat storageFormat, SchemaTableNam insertData(tableHandle, CREATE_TABLE_PARTITIONED_DATA, SESSION); // verify partitions were created - List partitionNames = getMetastoreClient(tableName.getSchemaName()).getPartitionNames(tableName.getSchemaName(), tableName.getTableName()) + List partitionNames = getMetastoreClient(tableName.getSchemaName()).getPartitionNames(SESSION.getUser(), tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new PrestoException(HIVE_METASTORE_ERROR, "Partition metadata not available")); assertEqualsIgnoreOrder(partitionNames, CREATE_TABLE_PARTITIONED_DATA.getMaterializedRows().stream() .map(row -> "ds=" + row.getField(CREATE_TABLE_PARTITIONED_DATA.getTypes().size() - 1)) @@ -1800,7 +1800,7 @@ private void doMetadataDelete(HiveStorageFormat storageFormat, SchemaTableName t } // verify partitions were created - List partitionNames = getMetastoreClient(tableName.getSchemaName()).getPartitionNames(tableName.getSchemaName(), tableName.getTableName()) + List partitionNames = getMetastoreClient(tableName.getSchemaName()).getPartitionNames(SESSION.getUser(), tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new PrestoException(HIVE_METASTORE_ERROR, "Partition metadata not available")); assertEqualsIgnoreOrder(partitionNames, CREATE_TABLE_PARTITIONED_DATA.getMaterializedRows().stream() .map(row -> "ds=" + row.getField(CREATE_TABLE_PARTITIONED_DATA.getTypes().size() - 1)) diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClientS3.java b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClientS3.java index 1bb81269e532..7869d77b9035 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClientS3.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClientS3.java @@ -325,7 +325,7 @@ private void doCreateTable(SchemaTableName tableName, HiveStorageFormat storageF // table, which fails without explicit configuration for S3. // We work around that by using a dummy location when creating the // table and update it here to the correct S3 location. - metastoreClient.updateTableLocation(database, tableName.getTableName(), outputHandle.getWritePath().get()); + metastoreClient.updateTableLocation(SESSION.getUser(), database, tableName.getTableName(), outputHandle.getWritePath().get()); // load the new table ConnectorTableHandle tableHandle = getTableHandle(tableName); @@ -352,7 +352,7 @@ private void doCreateTable(SchemaTableName tableName, HiveStorageFormat storageF private void dropTable(SchemaTableName table) { try { - metastoreClient.dropTable(table.getSchemaName(), table.getTableName()); + metastoreClient.dropTable(SESSION.getUser(), table.getSchemaName(), table.getTableName()); } catch (RuntimeException e) { // this usually occurs because the table was not created @@ -402,9 +402,9 @@ public TestingHiveMetastore(HiveCluster hiveCluster, ExecutorService executor, H } @Override - public Optional getDatabase(String databaseName) + public Optional getDatabase(String user, String databaseName) { - Optional database = super.getDatabase(databaseName); + Optional database = super.getDatabase(user, databaseName); if (database.isPresent()) { database.get().setLocationUri("s3://" + writableBucket + "/"); } @@ -412,18 +412,18 @@ public Optional getDatabase(String databaseName) } @Override - public void createTable(Table table) + public void createTable(String user, Table table) { // hack to work around the metastore not being configured for S3 table.getSd().setLocation("/"); - super.createTable(table); + super.createTable(user, table); } @Override - public void dropTable(String databaseName, String tableName) + public void dropTable(String user, String databaseName, String tableName) { try { - Optional
table = getTable(databaseName, tableName); + Optional
table = getTable(user, databaseName, tableName); if (!table.isPresent()) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } @@ -449,10 +449,10 @@ public void dropTable(String databaseName, String tableName) } } - public void updateTableLocation(String databaseName, String tableName, String location) + public void updateTableLocation(String user, String databaseName, String tableName, String location) { try { - Optional
table = getTable(databaseName, tableName); + Optional
table = getTable(user, databaseName, tableName); if (!table.isPresent()) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/metastore/InMemoryHiveMetastore.java b/presto-hive/src/test/java/com/facebook/presto/hive/metastore/InMemoryHiveMetastore.java index fa8a81344c20..56bb639c7f33 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/metastore/InMemoryHiveMetastore.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/metastore/InMemoryHiveMetastore.java @@ -88,13 +88,13 @@ public void createDatabase(Database database) } @Override - public List getAllDatabases() + public List getAllDatabases(String user) { return ImmutableList.copyOf(databases.keySet()); } @Override - public void createTable(Table table) + public void createTable(String user1, Table table) { SchemaTableName schemaTableName = new SchemaTableName(table.getDbName(), table.getTableName()); Table tableCopy = table.deepCopy(); @@ -137,7 +137,7 @@ else if (tableCopy.getSd().getLocation() != null) { } @Override - public void dropTable(String databaseName, String tableName) + public void dropTable(String user, String databaseName, String tableName) { SchemaTableName schemaTableName = new SchemaTableName(databaseName, tableName); Table table = relations.remove(schemaTableName); @@ -159,7 +159,7 @@ public void dropTable(String databaseName, String tableName) } @Override - public void alterTable(String databaseName, String tableName, Table newTable) + public void alterTable(String user, String databaseName, String tableName, Table newTable) { SchemaTableName oldName = new SchemaTableName(databaseName, tableName); SchemaTableName newName = new SchemaTableName(newTable.getDbName(), newTable.getTableName()); @@ -186,7 +186,7 @@ public void alterTable(String databaseName, String tableName, Table newTable) } @Override - public Optional> getAllTables(String databaseName) + public Optional> getAllTables(String user, String databaseName) { ImmutableList.Builder tables = ImmutableList.builder(); for (SchemaTableName schemaTableName : this.relations.keySet()) { @@ -198,7 +198,7 @@ public Optional> getAllTables(String databaseName) } @Override - public Optional> getAllViews(String databaseName) + public Optional> getAllViews(String user, String databaseName) { ImmutableList.Builder tables = ImmutableList.builder(); for (SchemaTableName schemaTableName : this.views.keySet()) { @@ -210,15 +210,15 @@ public Optional> getAllViews(String databaseName) } @Override - public Optional getDatabase(String databaseName) + public Optional getDatabase(String user, String databaseName) { return Optional.ofNullable(databases.get(databaseName)); } @Override - public void addPartitions(String databaseName, String tableName, List partitions) + public void addPartitions(String user, String databaseName, String tableName, List partitions) { - Optional
table = getTable(databaseName, tableName); + Optional
table = getTable(user, databaseName, tableName); if (!table.isPresent()) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } @@ -233,7 +233,7 @@ public void addPartitions(String databaseName, String tableName, List } @Override - public void dropPartition(String databaseName, String tableName, List parts) + public void dropPartition(String user, String databaseName, String tableName, List parts) { for (Entry entry : partitions.entrySet()) { PartitionName partitionName = entry.getKey(); @@ -245,7 +245,7 @@ public void dropPartition(String databaseName, String tableName, List pa } @Override - public void dropPartitionByName(String databaseName, String tableName, String partitionName) + public void dropPartitionByName(String user, String databaseName, String tableName, String partitionName) { for (PartitionName partition : partitions.keySet()) { if (partition.matches(databaseName, tableName, partitionName)) { @@ -255,7 +255,7 @@ public void dropPartitionByName(String databaseName, String tableName, String pa } @Override - public Optional> getPartitionNames(String databaseName, String tableName) + public Optional> getPartitionNames(String user, String databaseName, String tableName) { return Optional.of(ImmutableList.copyOf(partitions.entrySet().stream() .filter(entry -> entry.getKey().matches(databaseName, tableName)) @@ -264,7 +264,7 @@ public Optional> getPartitionNames(String databaseName, String tabl } @Override - public Optional getPartition(String databaseName, String tableName, String partitionName) + public Optional getPartition(String user, String databaseName, String tableName, String partitionName) { PartitionName name = new PartitionName(databaseName, tableName, partitionName); Partition partition = partitions.get(name); @@ -275,7 +275,7 @@ public Optional getPartition(String databaseName, String tableName, S } @Override - public Optional> getPartitionNamesByParts(String databaseName, String tableName, List parts) + public Optional> getPartitionNamesByParts(String user, String databaseName, String tableName, List parts) { return Optional.of(partitions.entrySet().stream() .filter(entry -> partitionMatches(entry.getValue(), databaseName, tableName, parts)) @@ -303,7 +303,7 @@ private static boolean partitionMatches(Partition partition, String databaseName } @Override - public Optional> getPartitionsByNames(String databaseName, String tableName, List partitionNames) + public Optional> getPartitionsByNames(String user, String databaseName, String tableName, List partitionNames) { ImmutableMap.Builder builder = ImmutableMap.builder(); for (String name : partitionNames) { @@ -318,7 +318,7 @@ public Optional> getPartitionsByNames(String databaseName } @Override - public Optional
getTable(String databaseName, String tableName) + public Optional
getTable(String user, String databaseName, String tableName) { SchemaTableName schemaTableName = new SchemaTableName(databaseName, tableName); return Optional.ofNullable(relations.get(schemaTableName)); diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/metastore/TestCachingHiveMetastore.java b/presto-hive/src/test/java/com/facebook/presto/hive/metastore/TestCachingHiveMetastore.java index b0643d13a7e8..ac55e00933fa 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/metastore/TestCachingHiveMetastore.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/metastore/TestCachingHiveMetastore.java @@ -22,6 +22,7 @@ import java.util.concurrent.TimeUnit; +import static com.facebook.presto.hive.HiveTestUtils.SESSION; import static com.facebook.presto.hive.metastore.MockHiveMetastoreClient.BAD_DATABASE; import static com.facebook.presto.hive.metastore.MockHiveMetastoreClient.TEST_DATABASE; import static com.facebook.presto.hive.metastore.MockHiveMetastoreClient.TEST_PARTITION1; @@ -55,14 +56,14 @@ public void testGetAllDatabases() throws Exception { assertEquals(mockClient.getAccessCount(), 0); - assertEquals(metastore.getAllDatabases(), ImmutableList.of(TEST_DATABASE)); + assertEquals(metastore.getAllDatabases(SESSION.getUser()), ImmutableList.of(TEST_DATABASE)); assertEquals(mockClient.getAccessCount(), 1); - assertEquals(metastore.getAllDatabases(), ImmutableList.of(TEST_DATABASE)); + assertEquals(metastore.getAllDatabases(SESSION.getUser()), ImmutableList.of(TEST_DATABASE)); assertEquals(mockClient.getAccessCount(), 1); metastore.flushCache(); - assertEquals(metastore.getAllDatabases(), ImmutableList.of(TEST_DATABASE)); + assertEquals(metastore.getAllDatabases(SESSION.getUser()), ImmutableList.of(TEST_DATABASE)); assertEquals(mockClient.getAccessCount(), 2); } @@ -71,21 +72,21 @@ public void testGetAllTable() throws Exception { assertEquals(mockClient.getAccessCount(), 0); - assertEquals(metastore.getAllTables(TEST_DATABASE).get(), ImmutableList.of(TEST_TABLE)); + assertEquals(metastore.getAllTables(SESSION.getUser(), TEST_DATABASE).get(), ImmutableList.of(TEST_TABLE)); assertEquals(mockClient.getAccessCount(), 1); - assertEquals(metastore.getAllTables(TEST_DATABASE).get(), ImmutableList.of(TEST_TABLE)); + assertEquals(metastore.getAllTables(SESSION.getUser(), TEST_DATABASE).get(), ImmutableList.of(TEST_TABLE)); assertEquals(mockClient.getAccessCount(), 1); metastore.flushCache(); - assertEquals(metastore.getAllTables(TEST_DATABASE).get(), ImmutableList.of(TEST_TABLE)); + assertEquals(metastore.getAllTables(SESSION.getUser(), TEST_DATABASE).get(), ImmutableList.of(TEST_TABLE)); assertEquals(mockClient.getAccessCount(), 2); } public void testInvalidDbGetAllTAbles() throws Exception { - assertFalse(metastore.getAllTables(BAD_DATABASE).isPresent()); + assertFalse(metastore.getAllTables(SESSION.getUser(), BAD_DATABASE).isPresent()); } @Test @@ -93,21 +94,21 @@ public void testGetTable() throws Exception { assertEquals(mockClient.getAccessCount(), 0); - assertNotNull(metastore.getTable(TEST_DATABASE, TEST_TABLE)); + assertNotNull(metastore.getTable(SESSION.getUser(), TEST_DATABASE, TEST_TABLE)); assertEquals(mockClient.getAccessCount(), 1); - assertNotNull(metastore.getTable(TEST_DATABASE, TEST_TABLE)); + assertNotNull(metastore.getTable(SESSION.getUser(), TEST_DATABASE, TEST_TABLE)); assertEquals(mockClient.getAccessCount(), 1); metastore.flushCache(); - assertNotNull(metastore.getTable(TEST_DATABASE, TEST_TABLE)); + assertNotNull(metastore.getTable(SESSION.getUser(), TEST_DATABASE, TEST_TABLE)); assertEquals(mockClient.getAccessCount(), 2); } public void testInvalidDbGetTable() throws Exception { - assertFalse(metastore.getTable(BAD_DATABASE, TEST_TABLE).isPresent()); + assertFalse(metastore.getTable(SESSION.getUser(), BAD_DATABASE, TEST_TABLE).isPresent()); } @Test @@ -116,14 +117,14 @@ public void testGetPartitionNames() { ImmutableList expectedPartitions = ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2); assertEquals(mockClient.getAccessCount(), 0); - assertEquals(metastore.getPartitionNames(TEST_DATABASE, TEST_TABLE).get(), expectedPartitions); + assertEquals(metastore.getPartitionNames(SESSION.getUser(), TEST_DATABASE, TEST_TABLE).get(), expectedPartitions); assertEquals(mockClient.getAccessCount(), 1); - assertEquals(metastore.getPartitionNames(TEST_DATABASE, TEST_TABLE).get(), expectedPartitions); + assertEquals(metastore.getPartitionNames(SESSION.getUser(), TEST_DATABASE, TEST_TABLE).get(), expectedPartitions); assertEquals(mockClient.getAccessCount(), 1); metastore.flushCache(); - assertEquals(metastore.getPartitionNames(TEST_DATABASE, TEST_TABLE).get(), expectedPartitions); + assertEquals(metastore.getPartitionNames(SESSION.getUser(), TEST_DATABASE, TEST_TABLE).get(), expectedPartitions); assertEquals(mockClient.getAccessCount(), 2); } @@ -131,7 +132,7 @@ public void testGetPartitionNames() public void testInvalidGetPartitionNames() throws Exception { - assertEquals(metastore.getPartitionNames(BAD_DATABASE, TEST_TABLE).get(), ImmutableList.of()); + assertEquals(metastore.getPartitionNames(SESSION.getUser(), BAD_DATABASE, TEST_TABLE).get(), ImmutableList.of()); } @Test @@ -142,14 +143,14 @@ public void testGetPartitionNamesByParts() ImmutableList expectedPartitions = ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2); assertEquals(mockClient.getAccessCount(), 0); - assertEquals(metastore.getPartitionNamesByParts(TEST_DATABASE, TEST_TABLE, parts).get(), expectedPartitions); + assertEquals(metastore.getPartitionNamesByParts(SESSION.getUser(), TEST_DATABASE, TEST_TABLE, parts).get(), expectedPartitions); assertEquals(mockClient.getAccessCount(), 1); - assertEquals(metastore.getPartitionNamesByParts(TEST_DATABASE, TEST_TABLE, parts).get(), expectedPartitions); + assertEquals(metastore.getPartitionNamesByParts(SESSION.getUser(), TEST_DATABASE, TEST_TABLE, parts).get(), expectedPartitions); assertEquals(mockClient.getAccessCount(), 1); metastore.flushCache(); - assertEquals(metastore.getPartitionNamesByParts(TEST_DATABASE, TEST_TABLE, parts).get(), expectedPartitions); + assertEquals(metastore.getPartitionNamesByParts(SESSION.getUser(), TEST_DATABASE, TEST_TABLE, parts).get(), expectedPartitions); assertEquals(mockClient.getAccessCount(), 2); } @@ -157,7 +158,7 @@ public void testInvalidGetPartitionNamesByParts() throws Exception { ImmutableList parts = ImmutableList.of(); - assertFalse(metastore.getPartitionNamesByParts(BAD_DATABASE, TEST_TABLE, parts).isPresent()); + assertFalse(metastore.getPartitionNamesByParts(SESSION.getUser(), BAD_DATABASE, TEST_TABLE, parts).isPresent()); } @Test @@ -165,35 +166,35 @@ public void testGetPartitionsByNames() throws Exception { assertEquals(mockClient.getAccessCount(), 0); - metastore.getTable(TEST_DATABASE, TEST_TABLE); + metastore.getTable(SESSION.getUser(), TEST_DATABASE, TEST_TABLE); assertEquals(mockClient.getAccessCount(), 1); // Select half of the available partitions and load them into the cache - assertEquals(metastore.getPartitionsByNames(TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1)).get().size(), 1); + assertEquals(metastore.getPartitionsByNames(SESSION.getUser(), TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1)).get().size(), 1); assertEquals(mockClient.getAccessCount(), 2); // Now select all of the partitions - assertEquals(metastore.getPartitionsByNames(TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2)).get().size(), 2); + assertEquals(metastore.getPartitionsByNames(SESSION.getUser(), TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2)).get().size(), 2); // There should be one more access to fetch the remaining partition assertEquals(mockClient.getAccessCount(), 3); // Now if we fetch any or both of them, they should not hit the client - assertEquals(metastore.getPartitionsByNames(TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1)).get().size(), 1); - assertEquals(metastore.getPartitionsByNames(TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION2)).get().size(), 1); - assertEquals(metastore.getPartitionsByNames(TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2)).get().size(), 2); + assertEquals(metastore.getPartitionsByNames(SESSION.getUser(), TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1)).get().size(), 1); + assertEquals(metastore.getPartitionsByNames(SESSION.getUser(), TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION2)).get().size(), 1); + assertEquals(metastore.getPartitionsByNames(SESSION.getUser(), TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2)).get().size(), 2); assertEquals(mockClient.getAccessCount(), 3); metastore.flushCache(); // Fetching both should only result in one batched access - assertEquals(metastore.getPartitionsByNames(TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2)).get().size(), 2); + assertEquals(metastore.getPartitionsByNames(SESSION.getUser(), TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2)).get().size(), 2); assertEquals(mockClient.getAccessCount(), 4); } public void testInvalidGetPartitionsByNames() throws Exception { - assertFalse(metastore.getPartitionsByNames(BAD_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1)).isPresent()); + assertFalse(metastore.getPartitionsByNames(SESSION.getUser(), BAD_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1)).isPresent()); } @Test @@ -203,7 +204,7 @@ public void testNoCacheExceptions() // Throw exceptions on usage mockClient.setThrowException(true); try { - metastore.getAllDatabases(); + metastore.getAllDatabases(SESSION.getUser()); } catch (RuntimeException ignored) { } @@ -211,7 +212,7 @@ public void testNoCacheExceptions() // Second try should hit the client again try { - metastore.getAllDatabases(); + metastore.getAllDatabases(SESSION.getUser()); } catch (RuntimeException ignored) { } From bae9de697343a0fbdd15e01b7fb43a0f2583a7a6 Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Mon, 26 Oct 2015 13:23:31 -0700 Subject: [PATCH 014/331] Revert 972d5d166bc8174c9cef50b1066f6efc9a07a02b --- .../parquet/ParquetRecordCursorProvider.java | 36 ++++++++----------- 1 file changed, 14 insertions(+), 22 deletions(-) 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 3854449498da..eb6bb880bed3 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 @@ -27,12 +27,10 @@ import com.google.common.collect.ImmutableSet; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.security.UserGroupInformation; import org.joda.time.DateTimeZone; import javax.inject.Inject; -import java.security.PrivilegedExceptionAction; import java.util.List; import java.util.Optional; import java.util.Properties; @@ -90,26 +88,20 @@ public Optional createHiveRecordCursor( throw new IllegalArgumentException("Can not read Parquet column: " + unsupportedColumns); } - UserGroupInformation ugi = UserGroupInformation.createRemoteUser(session.getUser()); - try { - return ugi.doAs((PrivilegedExceptionAction>) () -> Optional.of(new ParquetHiveRecordCursor( - configuration, - path, - start, - length, - schema, - partitionKeys, - columns, - useParquetColumnNames, - hiveStorageTimeZone, - typeManager, - isParquetPredicatePushdownEnabled(session), - effectivePredicate - ))); - } - catch (Exception e) { - throw new RuntimeException(e); - } + return Optional.of(new ParquetHiveRecordCursor( + configuration, + path, + start, + length, + schema, + partitionKeys, + columns, + useParquetColumnNames, + hiveStorageTimeZone, + typeManager, + isParquetPredicatePushdownEnabled(session), + effectivePredicate + )); } private static Predicate isParquetSupportedType() From a39315881019902b237d11540219a1c94b7ecc0a Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Mon, 26 Oct 2015 13:24:09 -0700 Subject: [PATCH 015/331] Load splits as user so that hdfs reads happen as that user. --- .../com/facebook/presto/hive/BackgroundHiveSplitLoader.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) 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 82defdecc369..557544a96b6b 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 @@ -38,8 +38,10 @@ import org.apache.hadoop.mapred.InputFormat; import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.security.UserGroupInformation; import java.io.IOException; +import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.Collections; import java.util.Deque; @@ -162,8 +164,9 @@ public TaskStatus process() try { CompletableFuture future; taskExecutionLock.readLock().lock(); + UserGroupInformation ugi = UserGroupInformation.createRemoteUser(session.getUser()); try { - future = loadSplits(); + future = ugi.doAs((PrivilegedExceptionAction>) BackgroundHiveSplitLoader.this::loadSplits); } finally { taskExecutionLock.readLock().unlock(); From 1b65e2fad9ac91da587e0193b114933461b7fcb6 Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Mon, 26 Oct 2015 13:26:36 -0700 Subject: [PATCH 016/331] UserBasedHiveMetastore to access metastore as user issuing the query. --- .../presto/hive/HiveClientModule.java | 6 +- .../hive/ThriftHiveMetastoreClient.java | 5 + .../metastore/UserBasedHiveMetastore.java | 608 ++++++++++++++++++ .../UserBasedHiveMetastoreStats.java | 141 ++++ 4 files changed, 757 insertions(+), 3 deletions(-) create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/metastore/UserBasedHiveMetastore.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/metastore/UserBasedHiveMetastoreStats.java 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 bc48cbfc7f0f..de95fec6e6fb 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 @@ -13,8 +13,8 @@ */ package com.facebook.presto.hive; -import com.facebook.presto.hive.metastore.CachingHiveMetastore; import com.facebook.presto.hive.metastore.HiveMetastore; +import com.facebook.presto.hive.metastore.UserBasedHiveMetastore; import com.facebook.presto.hive.orc.DwrfPageSourceFactory; import com.facebook.presto.hive.orc.DwrfRecordCursorProvider; import com.facebook.presto.hive.orc.OrcPageSourceFactory; @@ -81,9 +81,9 @@ public void configure(Binder binder) binder.bind(HiveMetastore.class).toInstance(metastore); } else { - binder.bind(HiveMetastore.class).to(CachingHiveMetastore.class).in(Scopes.SINGLETON); + binder.bind(HiveMetastore.class).to(UserBasedHiveMetastore.class).in(Scopes.SINGLETON); newExporter(binder).export(HiveMetastore.class) - .as(generatedNameOf(CachingHiveMetastore.class, connectorId)); + .as(generatedNameOf(UserBasedHiveMetastore.class, connectorId)); } binder.bind(NamenodeStats.class).in(Scopes.SINGLETON); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveMetastoreClient.java b/presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveMetastoreClient.java index d97aaf3d464f..9ab9986b7bab 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveMetastoreClient.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveMetastoreClient.java @@ -49,6 +49,11 @@ public ThriftHiveMetastoreClient(TProtocol protocol) this.client = new ThriftHiveMetastore.Client(protocol); } + public ThriftHiveMetastore.Client getClient() + { + return client; + } + @Override public void close() { diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/UserBasedHiveMetastore.java b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/UserBasedHiveMetastore.java new file mode 100644 index 000000000000..251962a01cc9 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/UserBasedHiveMetastore.java @@ -0,0 +1,608 @@ +/* + * 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.metastore; + +import com.facebook.presto.hive.HiveCluster; +import com.facebook.presto.hive.HiveViewNotSupportedException; +import com.facebook.presto.hive.TableAlreadyExistsException; +import com.facebook.presto.hive.ThriftHiveMetastoreClient; +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.SchemaNotFoundException; +import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.spi.TableNotFoundException; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import org.apache.hadoop.hive.common.FileUtils; +import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.HiveObjectRef; +import org.apache.hadoop.hive.metastore.api.HiveObjectType; +import org.apache.hadoop.hive.metastore.api.InvalidObjectException; +import org.apache.hadoop.hive.metastore.api.InvalidOperationException; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet; +import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo; +import org.apache.hadoop.hive.metastore.api.Role; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.api.UnknownDBException; +import org.apache.thrift.TException; +import org.weakref.jmx.Flatten; +import org.weakref.jmx.Managed; + +import javax.inject.Inject; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.function.Function; + +import static com.facebook.presto.hive.HiveErrorCode.HIVE_METASTORE_ERROR; +import static com.facebook.presto.hive.HiveUtil.PRESTO_VIEW_FLAG; +import static com.facebook.presto.hive.HiveUtil.isPrestoView; +import static com.facebook.presto.hive.RetryDriver.retry; +import static com.facebook.presto.hive.metastore.HivePrivilege.OWNERSHIP; +import static com.facebook.presto.hive.metastore.HivePrivilege.parsePrivilege; +import static com.google.common.base.Preconditions.checkArgument; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.toSet; +import static org.apache.hadoop.hive.metastore.api.PrincipalType.USER; +import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.HIVE_FILTER_FIELD_PARAMS; + +public class UserBasedHiveMetastore + implements HiveMetastore +{ + private final UserBasedHiveMetastoreStats stats = new UserBasedHiveMetastoreStats(); + protected final HiveCluster clientProvider; + + @Inject + public UserBasedHiveMetastore(HiveCluster hiveCluster) + { + this.clientProvider = requireNonNull(hiveCluster, "hiveCluster is null"); + } + + @Managed + @Flatten + public UserBasedHiveMetastoreStats getStats() + { + return stats; + } + + protected Function getExceptionMapper() + { + return Function.identity(); + } + + @Override + public void createTable(String user, Table table) + { + try { + retry() + .exceptionMapper(getExceptionMapper()) + .stopOn(AlreadyExistsException.class, InvalidObjectException.class, MetaException.class, NoSuchObjectException.class) + .stopOnIllegalExceptions() + .run("createTable", stats.getCreateTable().wrap(() -> { + try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { + client.getClient().set_ugi(user, ImmutableList.of()); + client.createTable(table); + } + return null; + })); + } + catch (AlreadyExistsException e) { + throw new TableAlreadyExistsException(new SchemaTableName(table.getDbName(), table.getTableName())); + } + catch (NoSuchObjectException e) { + throw new SchemaNotFoundException(table.getDbName()); + } + catch (TException e) { + throw new PrestoException(HIVE_METASTORE_ERROR, e); + } + catch (Exception e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw Throwables.propagate(e); + } + } + + @Override + public void dropTable(String user, String databaseName, String tableName) + { + try { + retry() + .stopOn(org.apache.hadoop.hive.metastore.api.NoSuchObjectException.class) + .stopOnIllegalExceptions() + .run("dropTable", stats.getDropTable().wrap(() -> { + try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { + client.getClient().set_ugi(user, ImmutableList.of()); + client.dropTable(databaseName, tableName, true); + } + return null; + })); + } + catch (org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { + throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); + } + catch (TException e) { + throw new PrestoException(HIVE_METASTORE_ERROR, e); + } + catch (Exception e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw com.google.common.base.Throwables.propagate(e); + } + } + + @Override + public void alterTable(String user, String databaseName, String tableName, Table table) + { + try { + retry() + .exceptionMapper(getExceptionMapper()) + .stopOn(InvalidOperationException.class, MetaException.class) + .stopOnIllegalExceptions() + .run("alterTable", stats.getAlterTable().wrap(() -> { + try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { + Optional
source = getTable(user, databaseName, tableName); + if (!source.isPresent()) { + throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); + } + client.getClient().set_ugi(user, ImmutableList.of()); + client.alterTable(databaseName, tableName, table); + } + return null; + })); + } + catch (org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { + throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); + } + catch (InvalidOperationException | MetaException e) { + throw com.google.common.base.Throwables.propagate(e); + } + catch (TException e) { + throw new PrestoException(HIVE_METASTORE_ERROR, e); + } + catch (Exception e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw com.google.common.base.Throwables.propagate(e); + } + } + + @Override + public void flushCache() + { + } + + @Override + public List getAllDatabases(String user) + { + try { + return retry() + .stopOnIllegalExceptions() + .run("getAllDatabases", stats.getGetAllDatabases().wrap(() -> { + try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { + client.getClient().set_ugi(user, ImmutableList.of()); + return client.getAllDatabases(); + } + })); + } + catch (Exception e) { + throw new PrestoException(HIVE_METASTORE_ERROR, e); + } + } + + @Override + public Optional> getAllTables(String user, String databaseName) + { + Callable> getAllTables = stats.getGetAllTables().wrap(() -> { + try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { + client.getClient().set_ugi(user, ImmutableList.of()); + return client.getAllTables(databaseName); + } + }); + + Callable getDatabase = stats.getGetDatabase().wrap(() -> { + try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { + client.getClient().set_ugi(user, ImmutableList.of()); + client.getDatabase(databaseName); + return null; + } + }); + + try { + return retry() + .stopOn(org.apache.hadoop.hive.metastore.api.NoSuchObjectException.class) + .stopOnIllegalExceptions() + .run("getAllTables", () -> { + List tables = getAllTables.call(); + if (tables.isEmpty()) { + // Check to see if the database exists + getDatabase.call(); + } + return Optional.of(tables); + }); + } + catch (org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { + return Optional.empty(); + } + catch (Exception e) { + throw new PrestoException(HIVE_METASTORE_ERROR, e); + } + } + + @Override + public Optional> getAllViews(String user, String databaseName) + { + try { + return retry() + .stopOn(UnknownDBException.class) + .stopOnIllegalExceptions() + .run("getAllViews", stats.getAllViews().wrap(() -> { + try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { + String filter = HIVE_FILTER_FIELD_PARAMS + PRESTO_VIEW_FLAG + " = \"true\""; + client.getClient().set_ugi(user, ImmutableList.of()); + return Optional.of(client.getTableNamesByFilter(databaseName, filter)); + } + })); + } + catch (UnknownDBException e) { + return Optional.empty(); + } + catch (Exception e) { + throw new PrestoException(HIVE_METASTORE_ERROR, e); + } + } + + @Override + public Optional getDatabase(String user, String databaseName) + { + try { + return retry() + .stopOn(org.apache.hadoop.hive.metastore.api.NoSuchObjectException.class) + .stopOnIllegalExceptions() + .run("getDatabase", stats.getGetDatabase().wrap(() -> { + try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { + client.getClient().set_ugi(user, ImmutableList.of()); + return Optional.of(client.getDatabase(databaseName)); + } + })); + } + catch (org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { + return Optional.empty(); + } + catch (Exception e) { + throw new PrestoException(HIVE_METASTORE_ERROR, e); + } + } + + @Override + public void addPartitions(String user, String databaseName, String tableName, List partitions) + { + if (partitions.isEmpty()) { + return; + } + try { + retry() + .exceptionMapper(getExceptionMapper()) + .stopOn(org.apache.hadoop.hive.metastore.api.AlreadyExistsException.class, org.apache.hadoop.hive.metastore.api.InvalidObjectException.class, MetaException.class, org.apache.hadoop.hive.metastore.api.NoSuchObjectException.class, PrestoException.class) + .stopOnIllegalExceptions() + .run("addPartitions", stats.getAddPartitions().wrap(() -> { + try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { + client.getClient().set_ugi(user, ImmutableList.of()); + int partitionsAdded = client.addPartitions(partitions); + if (partitionsAdded != partitions.size()) { + throw new PrestoException(HIVE_METASTORE_ERROR, + format("Hive metastore only added %s of %s partitions", partitionsAdded, partitions.size())); + } + } + return null; + })); + } + catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException | org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { + // todo partition already exists exception + throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); + } + catch (TException e) { + throw new PrestoException(HIVE_METASTORE_ERROR, e); + } + catch (Exception e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw com.google.common.base.Throwables.propagate(e); + } + } + + @Override + public void dropPartition(String user, String databaseName, String tableName, List parts) + { + try { + retry() + .stopOn(org.apache.hadoop.hive.metastore.api.NoSuchObjectException.class, MetaException.class) + .stopOnIllegalExceptions() + .run("dropPartition", stats.getDropPartition().wrap(() -> { + try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { + client.getClient().set_ugi(user, ImmutableList.of()); + client.dropPartition(databaseName, tableName, parts, true); + } + return null; + })); + } + catch (org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { + throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); + } + catch (TException e) { + throw new PrestoException(HIVE_METASTORE_ERROR, e); + } + catch (Exception e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw com.google.common.base.Throwables.propagate(e); + } + } + + @Override + public void dropPartitionByName(String user, String databaseName, String tableName, String partitionName) + { + try { + retry() + .stopOn(org.apache.hadoop.hive.metastore.api.NoSuchObjectException.class, MetaException.class) + .stopOnIllegalExceptions() + .run("dropPartitionByName", stats.getDropPartitionByName().wrap(() -> { + try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { + // It is observed that: (examples below assumes a table with one partition column `ds`) + // * When a partition doesn't exist (e.g. ds=2015-09-99), this thrift call is a no-op. It doesn't throw any exception. + // * When a typo exists in partition column name (e.g. dxs=2015-09-01), this thrift call will delete ds=2015-09-01. + client.getClient().set_ugi(user, ImmutableList.of()); + client.dropPartitionByName(databaseName, tableName, partitionName, true); + } + return null; + })); + } + catch (org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { + throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); + } + catch (TException e) { + throw new PrestoException(HIVE_METASTORE_ERROR, e); + } + catch (Exception e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw com.google.common.base.Throwables.propagate(e); + } + } + + @Override + public Optional> getPartitionNames(String user, String databaseName, String tableName) + { + try { + return retry() + .stopOn(org.apache.hadoop.hive.metastore.api.NoSuchObjectException.class) + .stopOnIllegalExceptions() + .run("getPartitionNames", stats.getGetPartitionNames().wrap(() -> { + try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { + client.getClient().set_ugi(user, ImmutableList.of()); + return Optional.of(client.getPartitionNames(databaseName, tableName)); + } + })); + } + catch (org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { + return Optional.empty(); + } + catch (Exception e) { + throw new PrestoException(HIVE_METASTORE_ERROR, e); + } + } + + @Override + public Optional> getPartitionNamesByParts(String user, String databaseName, String tableName, List parts) + { + try { + return retry() + .stopOn(org.apache.hadoop.hive.metastore.api.NoSuchObjectException.class) + .stopOnIllegalExceptions() + .run("getPartitionNamesByParts", stats.getGetPartitionNamesPs().wrap(() -> { + try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { + client.getClient().set_ugi(user, ImmutableList.of()); + return Optional.of(client.getPartitionNamesFiltered(databaseName, tableName, parts)); + } + })); + } + catch (org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { + return Optional.empty(); + } + catch (Exception e) { + throw new PrestoException(HIVE_METASTORE_ERROR, e); + } + } + + @Override + public Optional getPartition(String user, String databaseName, String tableName, String partitionName) + { + requireNonNull(partitionName, "partitionName is null"); + try { + return retry() + .stopOn(org.apache.hadoop.hive.metastore.api.NoSuchObjectException.class) + .stopOnIllegalExceptions() + .run("getPartitionsByNames", stats.getGetPartitionByName().wrap(() -> { + try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { + client.getClient().set_ugi(user, ImmutableList.of()); + return Optional.of(client.getPartitionByName(databaseName, tableName, partitionName)); + } + })); + } + catch (org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { + return Optional.empty(); + } + catch (Exception e) { + throw new PrestoException(HIVE_METASTORE_ERROR, e); + } + } + + @Override + public Optional> getPartitionsByNames(String user, String databaseName, String tableName, List partitionNames) + { + requireNonNull(partitionNames, "partitionNames is null"); + checkArgument(!Iterables.isEmpty(partitionNames), "partitionNames is empty"); + + try { + return retry() + .stopOn(org.apache.hadoop.hive.metastore.api.NoSuchObjectException.class) + .stopOnIllegalExceptions() + .run("getPartitionsByNames", stats.getGetPartitionsByNames().wrap(() -> { + try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { + List partitionColumnNames = ImmutableList.copyOf(Warehouse.makeSpecFromName(partitionNames.get(0)).keySet()); + ImmutableMap.Builder partitions = ImmutableMap.builder(); + client.getClient().set_ugi(user, ImmutableList.of()); + for (Partition partition : client.getPartitionsByNames(databaseName, tableName, partitionNames)) { + String partitionId = FileUtils.makePartName(partitionColumnNames, partition.getValues(), null); + partitions.put(partitionId, partition); + } + return Optional.of(partitions.build()); + } + })); + } + catch (org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { + // assume none of the partitions in the batch are available + return Optional.empty(); + } + catch (Exception e) { + throw new PrestoException(HIVE_METASTORE_ERROR, e); + } + } + + @Override + public Optional
getTable(String user, String databaseName, String tableName) + { + try { + return retry() + .stopOn(org.apache.hadoop.hive.metastore.api.NoSuchObjectException.class, HiveViewNotSupportedException.class) + .stopOnIllegalExceptions() + .run("getTable", stats.getGetTable().wrap(() -> { + try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { + client.getClient().set_ugi(user, ImmutableList.of()); + Table table = client.getTable(databaseName, tableName); + if (table.getTableType().equals(TableType.VIRTUAL_VIEW.name()) && (!isPrestoView(table))) { + throw new HiveViewNotSupportedException(new SchemaTableName(databaseName, tableName)); + } + return Optional.of(table); + } + })); + } + catch (org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { + return Optional.empty(); + } + catch (Exception e) { + throw new PrestoException(HIVE_METASTORE_ERROR, e); + } + } + + @Override + public Set getRoles(String user) + { + try (HiveMetastoreClient client = clientProvider.createMetastoreClient()) { + List roles = client.listRoles(user, USER); + if (roles == null) { + return ImmutableSet.of(); + } + return ImmutableSet.copyOf(roles.stream() + .map(Role::getRoleName) + .collect(toSet())); + } + catch (TException e) { + throw new PrestoException(HIVE_METASTORE_ERROR, e); + } + } + + @Override + public Set getDatabasePrivileges(String user, String databaseName) + { + ImmutableSet.Builder privileges = ImmutableSet.builder(); + + if (isDatabaseOwner(user, databaseName)) { + privileges.add(OWNERSHIP); + } + privileges.addAll(getPrivileges(user, new HiveObjectRef(HiveObjectType.DATABASE, databaseName, null, null, null))); + + return privileges.build(); + } + + @Override + public Set getTablePrivileges(String user, String databaseName, String tableName) + { + ImmutableSet.Builder privileges = ImmutableSet.builder(); + + if (isTableOwner(user, databaseName, tableName)) { + privileges.add(OWNERSHIP); + } + privileges.addAll(getPrivileges(user, new HiveObjectRef(HiveObjectType.TABLE, databaseName, tableName, null, null))); + + return privileges.build(); + } + + private Set getPrivileges(String user, HiveObjectRef objectReference) + { + ImmutableSet.Builder privileges = ImmutableSet.builder(); + try (HiveMetastoreClient client = clientProvider.createMetastoreClient()) { + PrincipalPrivilegeSet privilegeSet = client.getPrivilegeSet(objectReference, user, null); + + if (privilegeSet != null) { + Map> userPrivileges = privilegeSet.getUserPrivileges(); + if (userPrivileges != null) { + privileges.addAll(toGrants(userPrivileges.get(user))); + } + for (List rolePrivileges : privilegeSet.getRolePrivileges().values()) { + privileges.addAll(toGrants(rolePrivileges)); + } + // We do not add the group permissions as Hive does not seem to process these + } + } + catch (TException e) { + throw new PrestoException(HIVE_METASTORE_ERROR, e); + } + + return privileges.build(); + } + + private static Set toGrants(List userGrants) + { + if (userGrants == null) { + return ImmutableSet.of(); + } + + ImmutableSet.Builder privileges = ImmutableSet.builder(); + for (PrivilegeGrantInfo userGrant : userGrants) { + privileges.addAll(parsePrivilege(userGrant)); + if (userGrant.isGrantOption()) { + privileges.add(HivePrivilege.GRANT); + } + } + return privileges.build(); + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/UserBasedHiveMetastoreStats.java b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/UserBasedHiveMetastoreStats.java new file mode 100644 index 000000000000..7778e2ba9010 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/UserBasedHiveMetastoreStats.java @@ -0,0 +1,141 @@ +/* + * 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.metastore; + +import org.weakref.jmx.Managed; +import org.weakref.jmx.Nested; + +public class UserBasedHiveMetastoreStats +{ + private final HiveMetastoreApiStats getAllDatabases = new HiveMetastoreApiStats(); + private final HiveMetastoreApiStats getDatabase = new HiveMetastoreApiStats(); + private final HiveMetastoreApiStats getAllTables = new HiveMetastoreApiStats(); + private final HiveMetastoreApiStats getAllViews = new HiveMetastoreApiStats(); + private final HiveMetastoreApiStats getTable = new HiveMetastoreApiStats(); + private final HiveMetastoreApiStats getPartitionNames = new HiveMetastoreApiStats(); + private final HiveMetastoreApiStats getPartitionNamesPs = new HiveMetastoreApiStats(); + private final HiveMetastoreApiStats getPartitionByName = new HiveMetastoreApiStats(); + private final HiveMetastoreApiStats getPartitionsByNames = new HiveMetastoreApiStats(); + private final HiveMetastoreApiStats createTable = new HiveMetastoreApiStats(); + private final HiveMetastoreApiStats dropTable = new HiveMetastoreApiStats(); + private final HiveMetastoreApiStats alterTable = new HiveMetastoreApiStats(); + private final HiveMetastoreApiStats addPartitions = new HiveMetastoreApiStats(); + private final HiveMetastoreApiStats dropPartition = new HiveMetastoreApiStats(); + private final HiveMetastoreApiStats dropPartitionByName = new HiveMetastoreApiStats(); + + @Managed + @Nested + public HiveMetastoreApiStats getGetAllDatabases() + { + return getAllDatabases; + } + + @Managed + @Nested + public HiveMetastoreApiStats getGetDatabase() + { + return getDatabase; + } + + @Managed + @Nested + public HiveMetastoreApiStats getGetAllTables() + { + return getAllTables; + } + + @Managed + @Nested + public HiveMetastoreApiStats getAllViews() + { + return getAllViews; + } + + @Managed + @Nested + public HiveMetastoreApiStats getGetTable() + { + return getTable; + } + + @Managed + @Nested + public HiveMetastoreApiStats getGetPartitionNames() + { + return getPartitionNames; + } + + @Managed + @Nested + public HiveMetastoreApiStats getGetPartitionNamesPs() + { + return getPartitionNamesPs; + } + + @Managed + @Nested + public HiveMetastoreApiStats getGetPartitionByName() + { + return getPartitionByName; + } + + @Managed + @Nested + public HiveMetastoreApiStats getGetPartitionsByNames() + { + return getPartitionsByNames; + } + + @Managed + @Nested + public HiveMetastoreApiStats getCreateTable() + { + return createTable; + } + + @Managed + @Nested + public HiveMetastoreApiStats getDropTable() + { + return dropTable; + } + + @Managed + @Nested + public HiveMetastoreApiStats getAlterTable() + { + return alterTable; + } + + @Managed + @Nested + public HiveMetastoreApiStats getAddPartitions() + { + return addPartitions; + } + + @Managed + @Nested + public HiveMetastoreApiStats getDropPartition() + { + return dropPartition; + } + + @Managed + @Nested + public HiveMetastoreApiStats getDropPartitionByName() + { + return dropPartitionByName; + } +} From 7cbc53a635331388b4fcfd6601b4cffc88aa8cbb Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Mon, 26 Oct 2015 15:03:42 -0700 Subject: [PATCH 017/331] Import missed import. From acfd0c75cc96fe1f4e289a69866b32bef538010c Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Mon, 26 Oct 2015 15:43:54 -0700 Subject: [PATCH 018/331] Use standard charset for UTF-8 instead of string. From a5a4f8cdaec61eb5172c9b7680780c46e3aba484 Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Tue, 3 Nov 2015 12:08:31 -0800 Subject: [PATCH 019/331] Pick username from unix system and disallow overriding it. --- .../main/java/com/facebook/presto/cli/ClientOptions.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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 32bf33079643..364958cd6065 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 @@ -17,6 +17,7 @@ import com.google.common.base.Splitter; import com.google.common.collect.ImmutableMap; 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; @@ -66,8 +67,9 @@ public class ClientOptions @Option(name = "--keystore-password", title = "keystore password", description = "Keystore password") public String keystorePassword; - @Option(name = "--user", title = "user", description = "Username") - public String user = System.getProperty("user.name"); + // Pick the user name for the logged in user. + // Do not let it be overridden by users. + public String user = new UnixSystem().getUsername(); @Option(name = "--source", title = "source", description = "Name of source making query") public String source = "presto-cli"; From e26fd783db6a07310faddf73470f83c3e1d6e845 Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Mon, 23 Nov 2015 10:53:27 -0800 Subject: [PATCH 020/331] Revert "Propagate user session information throughout hive metastore." This reverts commit d01400182645e82f733760a2d938273c49dd5a88. --- .../facebook/presto/hive/HiveMetadata.java | 85 +++++++++---------- .../facebook/presto/hive/HivePageSink.java | 11 +-- .../presto/hive/HivePageSinkProvider.java | 7 +- .../presto/hive/HivePartitionManager.java | 12 +-- .../presto/hive/HiveSplitManager.java | 7 +- .../facebook/presto/hive/HiveWriteUtils.java | 9 +- .../hive/metastore/CachingHiveMetastore.java | 30 +++---- .../presto/hive/metastore/HiveMetastore.java | 34 ++++---- .../presto/hive/AbstractTestHiveClient.java | 8 +- .../presto/hive/AbstractTestHiveClientS3.java | 20 ++--- .../hive/metastore/InMemoryHiveMetastore.java | 32 +++---- .../metastore/TestCachingHiveMetastore.java | 59 +++++++------ 12 files changed, 150 insertions(+), 164 deletions(-) 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 a3ad7180b9de..c8e48aa7e21b 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 @@ -231,14 +231,14 @@ public HivePartitionManager getPartitionManager() @Override public List listSchemaNames(ConnectorSession session) { - return metastore.getAllDatabases(session.getUser()); + return metastore.getAllDatabases(); } @Override public HiveTableHandle getTableHandle(ConnectorSession session, SchemaTableName tableName) { requireNonNull(tableName, "tableName is null"); - if (!metastore.getTable(session.getUser(), tableName.getSchemaName(), tableName.getTableName()).isPresent()) { + if (!metastore.getTable(tableName.getSchemaName(), tableName.getTableName()).isPresent()) { return null; } return new HiveTableHandle(connectorId, tableName.getSchemaName(), tableName.getTableName()); @@ -249,12 +249,12 @@ public ConnectorTableMetadata getTableMetadata(ConnectorSession session, Connect { requireNonNull(tableHandle, "tableHandle is null"); SchemaTableName tableName = schemaTableName(tableHandle); - return getTableMetadata(session, tableName); + return getTableMetadata(tableName); } - private ConnectorTableMetadata getTableMetadata(ConnectorSession session, SchemaTableName tableName) + private ConnectorTableMetadata getTableMetadata(SchemaTableName tableName) { - Optional
table = metastore.getTable(session.getUser(), tableName.getSchemaName(), tableName.getTableName()); + Optional
table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()); if (!table.isPresent() || table.get().getTableType().equals(TableType.VIRTUAL_VIEW.name())) { throw new TableNotFoundException(tableName); } @@ -294,7 +294,7 @@ public List listTables(ConnectorSession session, String schemaN { ImmutableList.Builder tableNames = ImmutableList.builder(); for (String schemaName : listSchemas(session, schemaNameOrNull)) { - for (String tableName : metastore.getAllTables(session.getUser(), schemaName).orElse(emptyList())) { + for (String tableName : metastore.getAllTables(schemaName).orElse(emptyList())) { tableNames.add(new SchemaTableName(schemaName, tableName)); } } @@ -313,7 +313,7 @@ private List listSchemas(ConnectorSession session, String schemaNameOrNu public ColumnHandle getSampleWeightColumnHandle(ConnectorSession session, ConnectorTableHandle tableHandle) { SchemaTableName tableName = schemaTableName(tableHandle); - Optional
table = metastore.getTable(session.getUser(), tableName.getSchemaName(), tableName.getTableName()); + Optional
table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()); if (!table.isPresent()) { throw new TableNotFoundException(tableName); } @@ -335,7 +335,7 @@ public boolean canCreateSampledTables(ConnectorSession session) public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) { SchemaTableName tableName = schemaTableName(tableHandle); - Optional
table = metastore.getTable(session.getUser(), tableName.getSchemaName(), tableName.getTableName()); + Optional
table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()); if (!table.isPresent()) { throw new TableNotFoundException(tableName); } @@ -356,7 +356,7 @@ public Map> listTableColumns(ConnectorSess ImmutableMap.Builder> columns = ImmutableMap.builder(); for (SchemaTableName tableName : listTables(session, prefix)) { try { - columns.put(tableName, getTableMetadata(session, tableName).getColumns()); + columns.put(tableName, getTableMetadata(tableName).getColumns()); } catch (HiveViewNotSupportedException e) { // view is not supported @@ -397,18 +397,17 @@ public void createTable(ConnectorSession session, ConnectorTableMetadata tableMe List partitionedBy = getPartitionedBy(tableMetadata.getProperties()); List columnHandles = getColumnHandles(connectorId, tableMetadata, ImmutableSet.copyOf(partitionedBy)); HiveStorageFormat hiveStorageFormat = getHiveStorageFormat(tableMetadata.getProperties()); - createTable(session, schemaName, tableName, tableMetadata.getOwner(), columnHandles, hiveStorageFormat, partitionedBy); + createTable(schemaName, tableName, tableMetadata.getOwner(), columnHandles, hiveStorageFormat, partitionedBy); } - public void createTable(ConnectorSession session, - String schemaName, + public void createTable(String schemaName, String tableName, String tableOwner, List columnHandles, HiveStorageFormat hiveStorageFormat, List partitionedBy) { - Path targetPath = getTableDefaultLocation(session, metastore, hdfsEnvironment, schemaName, tableName); + Path targetPath = getTableDefaultLocation(metastore, hdfsEnvironment, schemaName, tableName); // verify the target directory for the table if (pathExists(hdfsEnvironment, targetPath)) { @@ -416,11 +415,10 @@ public void createTable(ConnectorSession session, } createDirectory(hdfsEnvironment, targetPath); - createTable(session, schemaName, tableName, tableOwner, columnHandles, hiveStorageFormat, partitionedBy, targetPath); + createTable(schemaName, tableName, tableOwner, columnHandles, hiveStorageFormat, partitionedBy, targetPath); } - private Table createTable(ConnectorSession session, - String schemaName, + private Table createTable(String schemaName, String tableName, String tableOwner, List columnHandles, @@ -486,7 +484,7 @@ else if (!partitionColumnNames.contains(name)) { ImmutableMap.of(), ImmutableMap.of())); - metastore.createTable(session.getUser(), table); + metastore.createTable(table); return table; } @@ -498,7 +496,7 @@ public void addColumn(ConnectorSession session, ConnectorTableHandle tableHandle } HiveTableHandle handle = checkType(tableHandle, HiveTableHandle.class, "tableHandle"); - Optional
tableMetadata = metastore.getTable(session.getUser(), handle.getSchemaName(), handle.getTableName()); + Optional
tableMetadata = metastore.getTable(handle.getSchemaName(), handle.getTableName()); if (!tableMetadata.isPresent()) { throw new TableNotFoundException(handle.getSchemaTableName()); } @@ -511,7 +509,7 @@ public void addColumn(ConnectorSession session, ConnectorTableHandle tableHandle sd.setCols(columns.build()); table.setSd(sd); - metastore.alterTable(session.getUser(), handle.getSchemaName(), handle.getTableName(), table); + metastore.alterTable(handle.getSchemaName(), handle.getTableName(), table); } @Override @@ -523,7 +521,7 @@ public void renameColumn(ConnectorSession session, ConnectorTableHandle tableHan HiveTableHandle hiveTableHandle = checkType(tableHandle, HiveTableHandle.class, "tableHandle"); HiveColumnHandle sourceHandle = checkType(source, HiveColumnHandle.class, "columnHandle"); - Optional
tableMetadata = metastore.getTable(session.getUser(), hiveTableHandle.getSchemaName(), hiveTableHandle.getTableName()); + Optional
tableMetadata = metastore.getTable(hiveTableHandle.getSchemaName(), hiveTableHandle.getTableName()); if (!tableMetadata.isPresent()) { throw new TableNotFoundException(hiveTableHandle.getSchemaTableName()); } @@ -540,7 +538,7 @@ public void renameColumn(ConnectorSession session, ConnectorTableHandle tableHan } sd.setCols(columns.build()); table.setSd(sd); - metastore.alterTable(session.getUser(), hiveTableHandle.getSchemaName(), hiveTableHandle.getTableName(), table); + metastore.alterTable(hiveTableHandle.getSchemaName(), hiveTableHandle.getTableName(), table); } @Override @@ -552,14 +550,14 @@ public void renameTable(ConnectorSession session, ConnectorTableHandle tableHand HiveTableHandle handle = checkType(tableHandle, HiveTableHandle.class, "tableHandle"); SchemaTableName tableName = schemaTableName(tableHandle); - Optional
source = metastore.getTable(session.getUser(), handle.getSchemaName(), handle.getTableName()); + Optional
source = metastore.getTable(handle.getSchemaName(), handle.getTableName()); if (!source.isPresent()) { throw new TableNotFoundException(tableName); } Table table = source.get(); table.setDbName(newTableName.getSchemaName()); table.setTableName(newTableName.getTableName()); - metastore.alterTable(session.getUser(), handle.getSchemaName(), handle.getTableName(), table); + metastore.alterTable(handle.getSchemaName(), handle.getTableName(), table); } @Override @@ -572,7 +570,7 @@ public void dropTable(ConnectorSession session, ConnectorTableHandle tableHandle throw new PrestoException(PERMISSION_DENIED, "DROP TABLE is disabled in this Hive catalog"); } - Optional
target = metastore.getTable(session.getUser(), handle.getSchemaName(), handle.getTableName()); + Optional
target = metastore.getTable(handle.getSchemaName(), handle.getTableName()); if (!target.isPresent()) { throw new TableNotFoundException(tableName); } @@ -581,7 +579,7 @@ public void dropTable(ConnectorSession session, ConnectorTableHandle tableHandle if (!session.getUser().equals(table.getOwner())) { throw new PrestoException(PERMISSION_DENIED, format("Unable to drop table '%s': owner of the table is different from session user", table)); } - metastore.dropTable(session.getUser(), handle.getSchemaName(), handle.getTableName()); + metastore.dropTable(handle.getSchemaName(), handle.getTableName()); } @Override @@ -601,7 +599,7 @@ public HiveOutputTableHandle beginCreateTable(ConnectorSession session, Connecto List columnHandles = getColumnHandles(connectorId, tableMetadata, ImmutableSet.copyOf(partitionedBy)); - Path targetPath = getTableDefaultLocation(session, metastore, hdfsEnvironment, schemaName, tableName); + Path targetPath = getTableDefaultLocation(metastore, hdfsEnvironment, schemaName, tableName); // verify the target directory for the table if (pathExists(hdfsEnvironment, targetPath)) { @@ -635,7 +633,7 @@ public void commitCreateTable(ConnectorSession session, ConnectorOutputTableHand .map(partitionUpdateCodec::fromJson) .collect(toList()); - Path targetPath = getTableDefaultLocation(session, metastore, hdfsEnvironment, handle.getSchemaName(), handle.getTableName()); + Path targetPath = getTableDefaultLocation(metastore, hdfsEnvironment, handle.getSchemaName(), handle.getTableName()); Path writePath = new Path(handle.getWritePath().get()); // rename if using a temporary directory @@ -651,12 +649,11 @@ public void commitCreateTable(ConnectorSession session, ConnectorOutputTableHand renameDirectory(hdfsEnvironment, handle.getSchemaName(), handle.getTableName(), writePath, targetPath); } - PartitionCommitter partitionCommitter = new PartitionCommitter(session, handle.getSchemaName(), handle.getTableName(), metastore, PARTITION_COMMIT_BATCH_SIZE); + PartitionCommitter partitionCommitter = new PartitionCommitter(handle.getSchemaName(), handle.getTableName(), metastore, PARTITION_COMMIT_BATCH_SIZE); try { partitionUpdates = PartitionUpdate.mergePartitionUpdates(partitionUpdates); Table table = createTable( - session, handle.getSchemaName(), handle.getTableName(), handle.getTableOwner(), @@ -694,7 +691,7 @@ public HiveInsertTableHandle beginInsert(ConnectorSession session, ConnectorTabl verifyJvmTimeZone(); SchemaTableName tableName = schemaTableName(tableHandle); - Optional
table = metastore.getTable(session.getUser(), tableName.getSchemaName(), tableName.getTableName()); + Optional
table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()); if (!table.isPresent()) { throw new TableNotFoundException(tableName); } @@ -727,7 +724,6 @@ public HiveInsertTableHandle beginInsert(ConnectorSession session, ConnectorTabl private static class PartitionCommitter implements Closeable { - private final ConnectorSession session; private final String schemaName; private final String tableName; private final HiveMetastore metastore; @@ -735,9 +731,8 @@ private static class PartitionCommitter private final List batch; private final List createdPartitions = new ArrayList<>(); - public PartitionCommitter(ConnectorSession session, String schemaName, String tableName, HiveMetastore metastore, int batchSize) + public PartitionCommitter(String schemaName, String tableName, HiveMetastore metastore, int batchSize) { - this.session = session; this.schemaName = schemaName; this.tableName = tableName; this.metastore = metastore; @@ -771,7 +766,7 @@ public void abort() // drop created partitions for (Partition createdPartition : getCreatedPartitions()) { try { - metastore.dropPartition(session.getUser(), schemaName, tableName, createdPartition.getValues()); + metastore.dropPartition(schemaName, tableName, createdPartition.getValues()); } catch (Exception e) { log.error(e, "Error rolling back new partition '%s' in table '%s.%s", createdPartition.getValues(), schemaName, tableName); @@ -781,7 +776,7 @@ public void abort() private void addBatch() { - metastore.addPartitions(session.getUser(), schemaName, tableName, batch); + metastore.addPartitions(schemaName, tableName, batch); createdPartitions.addAll(batch); batch.clear(); } @@ -798,11 +793,11 @@ public void commitInsert(ConnectorSession session, ConnectorInsertTableHandle in .collect(toList()); HiveStorageFormat storageFormat = handle.getHiveStorageFormat(); - PartitionCommitter partitionCommitter = new PartitionCommitter(session, handle.getSchemaName(), handle.getTableName(), metastore, PARTITION_COMMIT_BATCH_SIZE); + PartitionCommitter partitionCommitter = new PartitionCommitter(handle.getSchemaName(), handle.getTableName(), metastore, PARTITION_COMMIT_BATCH_SIZE); try { partitionUpdates = PartitionUpdate.mergePartitionUpdates(partitionUpdates); - Optional
table = metastore.getTable(session.getUser(), handle.getSchemaName(), handle.getTableName()); + Optional
table = metastore.getTable(handle.getSchemaName(), handle.getTableName()); if (!table.isPresent()) { throw new TableNotFoundException(new SchemaTableName(handle.getSchemaName(), handle.getTableName())); } @@ -895,7 +890,7 @@ public void rollbackInsert(ConnectorSession session, ConnectorInsertTableHandle } // Otherwise, insert was directly into the target table and partitions, and all must be checked for temp files - Optional
table = metastore.getTable(session.getUser(), handle.getSchemaName(), handle.getTableName()); + Optional
table = metastore.getTable(handle.getSchemaName(), handle.getTableName()); if (!table.isPresent()) { log.error("Error rolling back insert into table %s.%s. Table was dropped during insert, and data directory may contain temporary data", handle.getSchemaName(), handle.getTableName()); return; @@ -909,10 +904,10 @@ public void rollbackInsert(ConnectorSession session, ConnectorInsertTableHandle // check every existing partition that is outside for the base directory if (!table.get().getPartitionKeys().isEmpty()) { - List partitionNames = metastore.getPartitionNames(session.getUser(), handle.getSchemaName(), handle.getTableName()) + List partitionNames = metastore.getPartitionNames(handle.getSchemaName(), handle.getTableName()) .orElse(ImmutableList.of()); for (List partitionNameBatch : Iterables.partition(partitionNames, 10)) { - metastore.getPartitionsByNames(session.getUser(), handle.getSchemaName(), handle.getTableName(), partitionNameBatch).orElse(ImmutableMap.of()).values().stream() + metastore.getPartitionsByNames(handle.getSchemaName(), handle.getTableName(), partitionNameBatch).orElse(ImmutableMap.of()).values().stream() .map(partition -> partition.getSd().getLocation()) .filter(location -> !location.startsWith(tableDirectory)) .forEach(locationsToClean::add); @@ -1140,7 +1135,7 @@ public void createView(ConnectorSession session, SchemaTableName viewName, Strin ImmutableMap.of())); try { - metastore.createTable(session.getUser(), table); + metastore.createTable(table); } catch (TableAlreadyExistsException e) { throw new ViewAlreadyExistsException(e.getTableName()); @@ -1156,7 +1151,7 @@ public void dropView(ConnectorSession session, SchemaTableName viewName) } try { - metastore.dropTable(session.getUser(), viewName.getSchemaName(), viewName.getTableName()); + metastore.dropTable(viewName.getSchemaName(), viewName.getTableName()); } catch (TableNotFoundException e) { throw new ViewNotFoundException(e.getTableName()); @@ -1168,7 +1163,7 @@ public List listViews(ConnectorSession session, String schemaNa { ImmutableList.Builder tableNames = ImmutableList.builder(); for (String schemaName : listSchemas(session, schemaNameOrNull)) { - for (String tableName : metastore.getAllViews(session.getUser(), schemaName).orElse(emptyList())) { + for (String tableName : metastore.getAllViews(schemaName).orElse(emptyList())) { tableNames.add(new SchemaTableName(schemaName, tableName)); } } @@ -1188,7 +1183,7 @@ public Map getViews(ConnectorSession s } for (SchemaTableName schemaTableName : tableNames) { - Optional
table = metastore.getTable(session.getUser(), schemaTableName.getSchemaName(), schemaTableName.getTableName()); + Optional
table = metastore.getTable(schemaTableName.getSchemaName(), schemaTableName.getTableName()); if (table.isPresent() && HiveUtil.isPrestoView(table.get())) { views.put(schemaTableName, new ConnectorViewDefinition( schemaTableName, @@ -1220,7 +1215,7 @@ public OptionalLong metadataDelete(ConnectorSession session, ConnectorTableHandl //for (HivePartition hivePartition : layoutHandle.getOrComputePartitions(this, session, tableHandle)) { for (HivePartition hivePartition : getOrComputePartitions(layoutHandle, session, tableHandle)) { - metastore.dropPartitionByName(session.getUser(), handle.getSchemaName(), handle.getTableName(), hivePartition.getPartitionId()); + metastore.dropPartitionByName(handle.getSchemaName(), handle.getTableName(), hivePartition.getPartitionId()); } // it is too expensive to determine the exact number of deleted rows return OptionalLong.empty(); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSink.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSink.java index 6444e05878f1..4e991d1547c4 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSink.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSink.java @@ -15,7 +15,6 @@ import com.facebook.presto.hive.metastore.HiveMetastore; import com.facebook.presto.spi.ConnectorPageSink; -import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.Page; import com.facebook.presto.spi.PageIndexer; import com.facebook.presto.spi.PageIndexerFactory; @@ -89,8 +88,6 @@ public class HivePageSink implements ConnectorPageSink { - private final ConnectorSession session; - private final String schemaName; private final String tableName; @@ -125,7 +122,6 @@ public class HivePageSink private HiveRecordWriter[] writers = new HiveRecordWriter[0]; public HivePageSink( - ConnectorSession session, String schemaName, String tableName, boolean isCreateTable, @@ -142,7 +138,6 @@ public HivePageSink( boolean immutablePartitions, JsonCodec partitionUpdateCodec) { - this.session = requireNonNull(session, "session is null"); this.schemaName = requireNonNull(schemaName, "schemaName is null"); this.tableName = requireNonNull(tableName, "tableName is null"); @@ -215,7 +210,7 @@ public HivePageSink( conf = new JobConf(hdfsEnvironment.getConfiguration(writePath.get())); } else { - Optional
table = metastore.getTable(session.getUser(), schemaName, tableName); + Optional
table = metastore.getTable(schemaName, tableName); if (!table.isPresent()) { throw new PrestoException(HIVE_INVALID_METADATA, format("Table %s.%s was dropped during insert", schemaName, tableName)); } @@ -292,7 +287,7 @@ private HiveRecordWriter createWriter(List partitionRow) // attempt to get the existing partition (if this is an existing partitioned table) Optional partition = Optional.empty(); if (!partitionRow.isEmpty() && table != null) { - partition = metastore.getPartition(session.getUser(), schemaName, tableName, partitionName); + partition = metastore.getPartition(schemaName, tableName, partitionName); } if (!partition.isPresent()) { @@ -308,7 +303,7 @@ private HiveRecordWriter createWriter(List partitionRow) .map(HiveType::toHiveType) .map(HiveType::getHiveTypeName) .collect(Collectors.joining(":"))); - target = getTableDefaultLocation(session, metastore, hdfsEnvironment, schemaName, tableName).toString(); + target = getTableDefaultLocation(metastore, hdfsEnvironment, schemaName, tableName).toString(); if (!partitionRow.isEmpty()) { // verify the target directory for the partition does not already exist diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSinkProvider.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSinkProvider.java index 313fe5f5592d..7cb88075d5f6 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSinkProvider.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSinkProvider.java @@ -64,20 +64,19 @@ public HivePageSinkProvider( public ConnectorPageSink createPageSink(ConnectorSession session, ConnectorOutputTableHandle tableHandle) { HiveWritableTableHandle handle = checkType(tableHandle, HiveOutputTableHandle.class, "tableHandle"); - return createPageSink(session, handle, true); + return createPageSink(handle, true); } @Override public ConnectorPageSink createPageSink(ConnectorSession session, ConnectorInsertTableHandle tableHandle) { HiveInsertTableHandle handle = checkType(tableHandle, HiveInsertTableHandle.class, "tableHandle"); - return createPageSink(session, handle, false); + return createPageSink(handle, false); } - private ConnectorPageSink createPageSink(ConnectorSession session, HiveWritableTableHandle handle, boolean isCreateTable) + private ConnectorPageSink createPageSink(HiveWritableTableHandle handle, boolean isCreateTable) { return new HivePageSink( - session, handle.getSchemaName(), handle.getTableName(), isCreateTable, diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java index b41f6cb702fc..37eb15d30025 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java @@ -100,7 +100,7 @@ public HivePartitionResult getPartitions(ConnectorSession session, HiveMetastore } SchemaTableName tableName = hiveTableHandle.getSchemaTableName(); - Table table = getTable(session, metastore, tableName); + Table table = getTable(metastore, tableName); Optional bucket = getHiveBucket(table, effectivePredicate.extractFixedValues()); TupleDomain compactEffectivePredicate = toCompactTupleDomain(effectivePredicate, domainCompactionThreshold); @@ -110,7 +110,7 @@ public HivePartitionResult getPartitions(ConnectorSession session, HiveMetastore } List partitionColumns = getPartitionKeyColumnHandles(connectorId, table); - List partitionNames = getFilteredPartitionNames(session, metastore, tableName, partitionColumns, effectivePredicate); + List partitionNames = getFilteredPartitionNames(metastore, tableName, partitionColumns, effectivePredicate); // do a final pass to filter based on fields that could not be used to filter the partitions ImmutableList.Builder partitions = ImmutableList.builder(); @@ -164,9 +164,9 @@ private Optional> parseValuesAndFilte return Optional.of(builder.build()); } - private Table getTable(ConnectorSession session, HiveMetastore metastore, SchemaTableName tableName) + private Table getTable(HiveMetastore metastore, SchemaTableName tableName) { - Optional
target = metastore.getTable(session.getUser(), tableName.getSchemaName(), tableName.getTableName()); + Optional
target = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()); if (!target.isPresent()) { throw new TableNotFoundException(tableName); } @@ -185,7 +185,7 @@ private Table getTable(ConnectorSession session, HiveMetastore metastore, Schema return table; } - private List getFilteredPartitionNames(ConnectorSession session, HiveMetastore metastore, SchemaTableName tableName, List partitionKeys, TupleDomain effectivePredicate) + private List getFilteredPartitionNames(HiveMetastore metastore, SchemaTableName tableName, List partitionKeys, TupleDomain effectivePredicate) { List filter = new ArrayList<>(); for (HiveColumnHandle partitionKey : partitionKeys) { @@ -217,7 +217,7 @@ else if ((value instanceof Boolean) || (value instanceof Double) || (value insta } // fetch the partition names - return metastore.getPartitionNamesByParts(session.getUser(), tableName.getSchemaName(), tableName.getTableName(), filter) + return metastore.getPartitionNamesByParts(tableName.getSchemaName(), tableName.getTableName(), filter) .orElseThrow(() -> new TableNotFoundException(tableName)); } 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 1506fc8f3c64..8f63692545bc 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 @@ -157,11 +157,11 @@ public ConnectorSplitSource getSplits(ConnectorSession session, ConnectorTableLa // sort partitions partitions = Ordering.natural().onResultOf(HivePartition::getPartitionId).reverse().sortedCopy(partitions); - Optional
table = metastore.getTable(session.getUser(), tableName.getSchemaName(), tableName.getTableName()); + Optional
table = metastore.getTable(tableName.getSchemaName(), tableName.getTableName()); if (!table.isPresent()) { throw new TableNotFoundException(tableName); } - Iterable hivePartitions = getPartitionMetadata(session, table.get(), tableName, partitions); + Iterable hivePartitions = getPartitionMetadata(table.get(), tableName, partitions); HiveSplitLoader hiveSplitLoader = new BackgroundHiveSplitLoader( connectorId, @@ -185,7 +185,7 @@ public ConnectorSplitSource getSplits(ConnectorSession session, ConnectorTableLa return splitSource; } - private Iterable getPartitionMetadata(ConnectorSession session, Table table, SchemaTableName tableName, List hivePartitions) + private Iterable getPartitionMetadata(Table table, SchemaTableName tableName, List hivePartitions) { if (hivePartitions.isEmpty()) { return ImmutableList.of(); @@ -201,7 +201,6 @@ private Iterable getPartitionMetadata(ConnectorSession se Iterable> partitionNameBatches = partitionExponentially(hivePartitions, minPartitionBatchSize, maxPartitionBatchSize); Iterable> partitionBatches = transform(partitionNameBatches, partitionBatch -> { Optional> batch = metastore.getPartitionsByNames( - session.getUser(), tableName.getSchemaName(), tableName.getTableName(), Lists.transform(partitionBatch, HivePartition::getPartitionId)); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java index 0ebaa8158583..58e7aa397a98 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java @@ -14,7 +14,6 @@ package com.facebook.presto.hive; import com.facebook.presto.hive.metastore.HiveMetastore; -import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.SchemaNotFoundException; import com.facebook.presto.spi.SchemaTableName; @@ -302,9 +301,9 @@ private static void checkWritable( } } - public static Path getTableDefaultLocation(ConnectorSession session, HiveMetastore metastore, HdfsEnvironment hdfsEnvironment, String schemaName, String tableName) + public static Path getTableDefaultLocation(HiveMetastore metastore, HdfsEnvironment hdfsEnvironment, String schemaName, String tableName) { - String location = getDatabase(session, metastore, schemaName).getLocationUri(); + String location = getDatabase(metastore, schemaName).getLocationUri(); if (isNullOrEmpty(location)) { throw new PrestoException(HIVE_DATABASE_LOCATION_ERROR, format("Database '%s' location is not set", schemaName)); } @@ -320,9 +319,9 @@ public static Path getTableDefaultLocation(ConnectorSession session, HiveMetasto return new Path(databasePath, tableName); } - private static Database getDatabase(ConnectorSession session, HiveMetastore metastore, String database) + private static Database getDatabase(HiveMetastore metastore, String database) { - return metastore.getDatabase(session.getUser(), database).orElseThrow(() -> new SchemaNotFoundException(database)); + return metastore.getDatabase(database).orElseThrow(() -> new SchemaNotFoundException(database)); } public static boolean pathExists(HdfsEnvironment hdfsEnvironment, Path path) 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 ed668aabb338..75d7074a543a 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 @@ -305,7 +305,7 @@ private static Map getAll(LoadingCache cache, Iterable key } @Override - public List getAllDatabases(String user) + public List getAllDatabases() { return get(databaseNamesCache, ""); } @@ -328,7 +328,7 @@ private List loadAllDatabases() } @Override - public Optional getDatabase(String user, String databaseName) + public Optional getDatabase(String databaseName) { return get(databaseCache, databaseName); } @@ -355,7 +355,7 @@ private Optional loadDatabase(String databaseName) } @Override - public Optional> getAllTables(String user, String databaseName) + public Optional> getAllTables(String databaseName) { return get(tableNamesCache, databaseName); } @@ -398,13 +398,13 @@ private Optional> loadAllTables(String databaseName) } @Override - public Optional
getTable(String user, String databaseName, String tableName) + public Optional
getTable(String databaseName, String tableName) { return get(tableCache, HiveTableName.table(databaseName, tableName)); } @Override - public Optional> getAllViews(String user, String databaseName) + public Optional> getAllViews(String databaseName) { return get(viewNamesCache, databaseName); } @@ -432,7 +432,7 @@ private Optional> loadAllViews(String databaseName) } @Override - public void createTable(String user, Table table) + public void createTable(Table table) { try { retry() @@ -467,7 +467,7 @@ public void createTable(String user, Table table) } @Override - public void dropTable(String user, String databaseName, String tableName) + public void dropTable(String databaseName, String tableName) { try { retry() @@ -506,7 +506,7 @@ protected void invalidateTable(String databaseName, String tableName) } @Override - public void alterTable(String user, String databaseName, String tableName, Table table) + public void alterTable(String databaseName, String tableName, Table table) { try { retry() @@ -570,7 +570,7 @@ private Optional
loadTable(HiveTableName hiveTableName) } @Override - public Optional> getPartitionNames(String user, String databaseName, String tableName) + public Optional> getPartitionNames(String databaseName, String tableName) { return get(partitionNamesCache, HiveTableName.table(databaseName, tableName)); } @@ -602,7 +602,7 @@ private Optional> loadPartitionNames(HiveTableName hiveTableName) } @Override - public Optional> getPartitionNamesByParts(String user, String databaseName, String tableName, List parts) + public Optional> getPartitionNamesByParts(String databaseName, String tableName, List parts) { return get(partitionFilterCache, PartitionFilter.partitionFilter(databaseName, tableName, parts)); } @@ -632,7 +632,7 @@ private Optional> loadPartitionNamesByParts(PartitionFilter partiti } @Override - public void addPartitions(String user, String databaseName, String tableName, List partitions) + public void addPartitions(String databaseName, String tableName, List partitions) { if (partitions.isEmpty()) { return; @@ -676,7 +676,7 @@ public void addPartitions(String user, String databaseName, String tableName, Li } @Override - public void dropPartition(String user, String databaseName, String tableName, List parts) + public void dropPartition(String databaseName, String tableName, List parts) { try { retry() @@ -707,7 +707,7 @@ public void dropPartition(String user, String databaseName, String tableName, Li } @Override - public void dropPartitionByName(String user, String databaseName, String tableName, String partitionName) + public void dropPartitionByName(String databaseName, String tableName, String partitionName) { try { retry() @@ -753,7 +753,7 @@ private void invalidatePartitionCache(String databaseName, String tableName) } @Override - public Optional> getPartitionsByNames(String user, String databaseName, String tableName, List partitionNames) + public Optional> getPartitionsByNames(String databaseName, String tableName, List partitionNames) { Iterable names = transform(partitionNames, name -> HivePartitionName.partition(databaseName, tableName, name)); @@ -769,7 +769,7 @@ public Optional> getPartitionsByNames(String user, String } @Override - public Optional getPartition(String user, String databaseName, String tableName, String partitionName) + public Optional getPartition(String databaseName, String tableName, String partitionName) { HivePartitionName name = HivePartitionName.partition(databaseName, tableName, partitionName); return get(partitionCache, name); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/HiveMetastore.java b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/HiveMetastore.java index 066cb9576a23..49e9b71da794 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/HiveMetastore.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/HiveMetastore.java @@ -30,43 +30,43 @@ public interface HiveMetastore { String DEFAULT_DATABASE_NAME = "default"; - void createTable(String user, Table table); + void createTable(Table table); - void dropTable(String user, String databaseName, String tableName); + void dropTable(String databaseName, String tableName); - void alterTable(String user, String databaseName, String tableName, Table table); + void alterTable(String databaseName, String tableName, Table table); @Managed void flushCache(); - List getAllDatabases(String user); + List getAllDatabases(); - Optional> getAllTables(String user, String databaseName); + Optional> getAllTables(String databaseName); - Optional> getAllViews(String user, String databaseName); + Optional> getAllViews(String databaseName); - Optional getDatabase(String user, String databaseName); + Optional getDatabase(String databaseName); /** * Adds partitions to the table in a single atomic task. The implementation * must either add all partitions and return normally, or add no partitions and * throw an exception. */ - void addPartitions(String user, String databaseName, String tableName, List partitions); + void addPartitions(String databaseName, String tableName, List partitions); - void dropPartition(String user, String databaseName, String tableName, List parts); + void dropPartition(String databaseName, String tableName, List parts); - void dropPartitionByName(String user, String databaseName, String tableName, String partitionName); + void dropPartitionByName(String databaseName, String tableName, String partitionName); - Optional> getPartitionNames(String user, String databaseName, String tableName); + Optional> getPartitionNames(String databaseName, String tableName); - Optional> getPartitionNamesByParts(String user, String databaseName, String tableName, List parts); + Optional> getPartitionNamesByParts(String databaseName, String tableName, List parts); - Optional getPartition(String user, String databaseName, String tableName, String partitionName); + Optional getPartition(String databaseName, String tableName, String partitionName); - Optional> getPartitionsByNames(String user, String databaseName, String tableName, List partitionNames); + Optional> getPartitionsByNames(String databaseName, String tableName, List partitionNames); - Optional
getTable(String user, String databaseName, String tableName); + Optional
getTable(String databaseName, String tableName); Set getRoles(String user); @@ -81,7 +81,7 @@ default boolean isDatabaseOwner(String user, String databaseName) return true; } - Optional databaseMetadata = getDatabase(user, databaseName); + Optional databaseMetadata = getDatabase(databaseName); if (!databaseMetadata.isPresent()) { return false; } @@ -101,7 +101,7 @@ default boolean isDatabaseOwner(String user, String databaseName) default boolean isTableOwner(String user, String databaseName, String tableName) { // a table can only be owned by a user - Optional
table = getTable(user, databaseName, tableName); + Optional
table = getTable(databaseName, tableName); return table.isPresent() && user.equals(table.get().getOwner()); } } 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 8a5692904b4c..1cd4adf8046d 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 @@ -1564,7 +1564,7 @@ protected void doCreateEmptyTable(SchemaTableName tableName, HiveStorageFormat s assertEquals(tableMetadata.getColumns(), createTableColumns); // verify table format - Table table = getMetastoreClient(tableName.getSchemaName()).getTable(SESSION.getUser(), tableName.getSchemaName(), tableName.getTableName()).get(); + Table table = getMetastoreClient(tableName.getSchemaName()).getTable(tableName.getSchemaName(), tableName.getTableName()).get(); if (!table.getSd().getInputFormat().equals(storageFormat.getInputFormat())) { assertEquals(table.getSd().getInputFormat(), storageFormat.getInputFormat()); } @@ -1678,7 +1678,7 @@ protected Set listAllDataFiles(ConnectorInsertTableHandle tableHandle) protected Set listAllDataFiles(String schemaName, String tableName) throws IOException { - Table table = metastoreClient.getTable(SESSION.getUser(), schemaName, tableName).get(); + Table table = metastoreClient.getTable(schemaName, tableName).get(); Path path = new Path(table.getSd().getLocation()); Set existingFiles = new HashSet<>(); return listAllDataFiles(path, existingFiles); @@ -1715,7 +1715,7 @@ private void doInsertPartitioned(HiveStorageFormat storageFormat, SchemaTableNam insertData(tableHandle, CREATE_TABLE_PARTITIONED_DATA, SESSION); // verify partitions were created - List partitionNames = getMetastoreClient(tableName.getSchemaName()).getPartitionNames(SESSION.getUser(), tableName.getSchemaName(), tableName.getTableName()) + List partitionNames = getMetastoreClient(tableName.getSchemaName()).getPartitionNames(tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new PrestoException(HIVE_METASTORE_ERROR, "Partition metadata not available")); assertEqualsIgnoreOrder(partitionNames, CREATE_TABLE_PARTITIONED_DATA.getMaterializedRows().stream() .map(row -> "ds=" + row.getField(CREATE_TABLE_PARTITIONED_DATA.getTypes().size() - 1)) @@ -1800,7 +1800,7 @@ private void doMetadataDelete(HiveStorageFormat storageFormat, SchemaTableName t } // verify partitions were created - List partitionNames = getMetastoreClient(tableName.getSchemaName()).getPartitionNames(SESSION.getUser(), tableName.getSchemaName(), tableName.getTableName()) + List partitionNames = getMetastoreClient(tableName.getSchemaName()).getPartitionNames(tableName.getSchemaName(), tableName.getTableName()) .orElseThrow(() -> new PrestoException(HIVE_METASTORE_ERROR, "Partition metadata not available")); assertEqualsIgnoreOrder(partitionNames, CREATE_TABLE_PARTITIONED_DATA.getMaterializedRows().stream() .map(row -> "ds=" + row.getField(CREATE_TABLE_PARTITIONED_DATA.getTypes().size() - 1)) diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClientS3.java b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClientS3.java index 7869d77b9035..1bb81269e532 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClientS3.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/AbstractTestHiveClientS3.java @@ -325,7 +325,7 @@ private void doCreateTable(SchemaTableName tableName, HiveStorageFormat storageF // table, which fails without explicit configuration for S3. // We work around that by using a dummy location when creating the // table and update it here to the correct S3 location. - metastoreClient.updateTableLocation(SESSION.getUser(), database, tableName.getTableName(), outputHandle.getWritePath().get()); + metastoreClient.updateTableLocation(database, tableName.getTableName(), outputHandle.getWritePath().get()); // load the new table ConnectorTableHandle tableHandle = getTableHandle(tableName); @@ -352,7 +352,7 @@ private void doCreateTable(SchemaTableName tableName, HiveStorageFormat storageF private void dropTable(SchemaTableName table) { try { - metastoreClient.dropTable(SESSION.getUser(), table.getSchemaName(), table.getTableName()); + metastoreClient.dropTable(table.getSchemaName(), table.getTableName()); } catch (RuntimeException e) { // this usually occurs because the table was not created @@ -402,9 +402,9 @@ public TestingHiveMetastore(HiveCluster hiveCluster, ExecutorService executor, H } @Override - public Optional getDatabase(String user, String databaseName) + public Optional getDatabase(String databaseName) { - Optional database = super.getDatabase(user, databaseName); + Optional database = super.getDatabase(databaseName); if (database.isPresent()) { database.get().setLocationUri("s3://" + writableBucket + "/"); } @@ -412,18 +412,18 @@ public Optional getDatabase(String user, String databaseName) } @Override - public void createTable(String user, Table table) + public void createTable(Table table) { // hack to work around the metastore not being configured for S3 table.getSd().setLocation("/"); - super.createTable(user, table); + super.createTable(table); } @Override - public void dropTable(String user, String databaseName, String tableName) + public void dropTable(String databaseName, String tableName) { try { - Optional
table = getTable(user, databaseName, tableName); + Optional
table = getTable(databaseName, tableName); if (!table.isPresent()) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } @@ -449,10 +449,10 @@ public void dropTable(String user, String databaseName, String tableName) } } - public void updateTableLocation(String user, String databaseName, String tableName, String location) + public void updateTableLocation(String databaseName, String tableName, String location) { try { - Optional
table = getTable(user, databaseName, tableName); + Optional
table = getTable(databaseName, tableName); if (!table.isPresent()) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/metastore/InMemoryHiveMetastore.java b/presto-hive/src/test/java/com/facebook/presto/hive/metastore/InMemoryHiveMetastore.java index 56bb639c7f33..fa8a81344c20 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/metastore/InMemoryHiveMetastore.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/metastore/InMemoryHiveMetastore.java @@ -88,13 +88,13 @@ public void createDatabase(Database database) } @Override - public List getAllDatabases(String user) + public List getAllDatabases() { return ImmutableList.copyOf(databases.keySet()); } @Override - public void createTable(String user1, Table table) + public void createTable(Table table) { SchemaTableName schemaTableName = new SchemaTableName(table.getDbName(), table.getTableName()); Table tableCopy = table.deepCopy(); @@ -137,7 +137,7 @@ else if (tableCopy.getSd().getLocation() != null) { } @Override - public void dropTable(String user, String databaseName, String tableName) + public void dropTable(String databaseName, String tableName) { SchemaTableName schemaTableName = new SchemaTableName(databaseName, tableName); Table table = relations.remove(schemaTableName); @@ -159,7 +159,7 @@ public void dropTable(String user, String databaseName, String tableName) } @Override - public void alterTable(String user, String databaseName, String tableName, Table newTable) + public void alterTable(String databaseName, String tableName, Table newTable) { SchemaTableName oldName = new SchemaTableName(databaseName, tableName); SchemaTableName newName = new SchemaTableName(newTable.getDbName(), newTable.getTableName()); @@ -186,7 +186,7 @@ public void alterTable(String user, String databaseName, String tableName, Table } @Override - public Optional> getAllTables(String user, String databaseName) + public Optional> getAllTables(String databaseName) { ImmutableList.Builder tables = ImmutableList.builder(); for (SchemaTableName schemaTableName : this.relations.keySet()) { @@ -198,7 +198,7 @@ public Optional> getAllTables(String user, String databaseName) } @Override - public Optional> getAllViews(String user, String databaseName) + public Optional> getAllViews(String databaseName) { ImmutableList.Builder tables = ImmutableList.builder(); for (SchemaTableName schemaTableName : this.views.keySet()) { @@ -210,15 +210,15 @@ public Optional> getAllViews(String user, String databaseName) } @Override - public Optional getDatabase(String user, String databaseName) + public Optional getDatabase(String databaseName) { return Optional.ofNullable(databases.get(databaseName)); } @Override - public void addPartitions(String user, String databaseName, String tableName, List partitions) + public void addPartitions(String databaseName, String tableName, List partitions) { - Optional
table = getTable(user, databaseName, tableName); + Optional
table = getTable(databaseName, tableName); if (!table.isPresent()) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } @@ -233,7 +233,7 @@ public void addPartitions(String user, String databaseName, String tableName, Li } @Override - public void dropPartition(String user, String databaseName, String tableName, List parts) + public void dropPartition(String databaseName, String tableName, List parts) { for (Entry entry : partitions.entrySet()) { PartitionName partitionName = entry.getKey(); @@ -245,7 +245,7 @@ public void dropPartition(String user, String databaseName, String tableName, Li } @Override - public void dropPartitionByName(String user, String databaseName, String tableName, String partitionName) + public void dropPartitionByName(String databaseName, String tableName, String partitionName) { for (PartitionName partition : partitions.keySet()) { if (partition.matches(databaseName, tableName, partitionName)) { @@ -255,7 +255,7 @@ public void dropPartitionByName(String user, String databaseName, String tableNa } @Override - public Optional> getPartitionNames(String user, String databaseName, String tableName) + public Optional> getPartitionNames(String databaseName, String tableName) { return Optional.of(ImmutableList.copyOf(partitions.entrySet().stream() .filter(entry -> entry.getKey().matches(databaseName, tableName)) @@ -264,7 +264,7 @@ public Optional> getPartitionNames(String user, String databaseName } @Override - public Optional getPartition(String user, String databaseName, String tableName, String partitionName) + public Optional getPartition(String databaseName, String tableName, String partitionName) { PartitionName name = new PartitionName(databaseName, tableName, partitionName); Partition partition = partitions.get(name); @@ -275,7 +275,7 @@ public Optional getPartition(String user, String databaseName, String } @Override - public Optional> getPartitionNamesByParts(String user, String databaseName, String tableName, List parts) + public Optional> getPartitionNamesByParts(String databaseName, String tableName, List parts) { return Optional.of(partitions.entrySet().stream() .filter(entry -> partitionMatches(entry.getValue(), databaseName, tableName, parts)) @@ -303,7 +303,7 @@ private static boolean partitionMatches(Partition partition, String databaseName } @Override - public Optional> getPartitionsByNames(String user, String databaseName, String tableName, List partitionNames) + public Optional> getPartitionsByNames(String databaseName, String tableName, List partitionNames) { ImmutableMap.Builder builder = ImmutableMap.builder(); for (String name : partitionNames) { @@ -318,7 +318,7 @@ public Optional> getPartitionsByNames(String user, String } @Override - public Optional
getTable(String user, String databaseName, String tableName) + public Optional
getTable(String databaseName, String tableName) { SchemaTableName schemaTableName = new SchemaTableName(databaseName, tableName); return Optional.ofNullable(relations.get(schemaTableName)); diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/metastore/TestCachingHiveMetastore.java b/presto-hive/src/test/java/com/facebook/presto/hive/metastore/TestCachingHiveMetastore.java index ac55e00933fa..b0643d13a7e8 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/metastore/TestCachingHiveMetastore.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/metastore/TestCachingHiveMetastore.java @@ -22,7 +22,6 @@ import java.util.concurrent.TimeUnit; -import static com.facebook.presto.hive.HiveTestUtils.SESSION; import static com.facebook.presto.hive.metastore.MockHiveMetastoreClient.BAD_DATABASE; import static com.facebook.presto.hive.metastore.MockHiveMetastoreClient.TEST_DATABASE; import static com.facebook.presto.hive.metastore.MockHiveMetastoreClient.TEST_PARTITION1; @@ -56,14 +55,14 @@ public void testGetAllDatabases() throws Exception { assertEquals(mockClient.getAccessCount(), 0); - assertEquals(metastore.getAllDatabases(SESSION.getUser()), ImmutableList.of(TEST_DATABASE)); + assertEquals(metastore.getAllDatabases(), ImmutableList.of(TEST_DATABASE)); assertEquals(mockClient.getAccessCount(), 1); - assertEquals(metastore.getAllDatabases(SESSION.getUser()), ImmutableList.of(TEST_DATABASE)); + assertEquals(metastore.getAllDatabases(), ImmutableList.of(TEST_DATABASE)); assertEquals(mockClient.getAccessCount(), 1); metastore.flushCache(); - assertEquals(metastore.getAllDatabases(SESSION.getUser()), ImmutableList.of(TEST_DATABASE)); + assertEquals(metastore.getAllDatabases(), ImmutableList.of(TEST_DATABASE)); assertEquals(mockClient.getAccessCount(), 2); } @@ -72,21 +71,21 @@ public void testGetAllTable() throws Exception { assertEquals(mockClient.getAccessCount(), 0); - assertEquals(metastore.getAllTables(SESSION.getUser(), TEST_DATABASE).get(), ImmutableList.of(TEST_TABLE)); + assertEquals(metastore.getAllTables(TEST_DATABASE).get(), ImmutableList.of(TEST_TABLE)); assertEquals(mockClient.getAccessCount(), 1); - assertEquals(metastore.getAllTables(SESSION.getUser(), TEST_DATABASE).get(), ImmutableList.of(TEST_TABLE)); + assertEquals(metastore.getAllTables(TEST_DATABASE).get(), ImmutableList.of(TEST_TABLE)); assertEquals(mockClient.getAccessCount(), 1); metastore.flushCache(); - assertEquals(metastore.getAllTables(SESSION.getUser(), TEST_DATABASE).get(), ImmutableList.of(TEST_TABLE)); + assertEquals(metastore.getAllTables(TEST_DATABASE).get(), ImmutableList.of(TEST_TABLE)); assertEquals(mockClient.getAccessCount(), 2); } public void testInvalidDbGetAllTAbles() throws Exception { - assertFalse(metastore.getAllTables(SESSION.getUser(), BAD_DATABASE).isPresent()); + assertFalse(metastore.getAllTables(BAD_DATABASE).isPresent()); } @Test @@ -94,21 +93,21 @@ public void testGetTable() throws Exception { assertEquals(mockClient.getAccessCount(), 0); - assertNotNull(metastore.getTable(SESSION.getUser(), TEST_DATABASE, TEST_TABLE)); + assertNotNull(metastore.getTable(TEST_DATABASE, TEST_TABLE)); assertEquals(mockClient.getAccessCount(), 1); - assertNotNull(metastore.getTable(SESSION.getUser(), TEST_DATABASE, TEST_TABLE)); + assertNotNull(metastore.getTable(TEST_DATABASE, TEST_TABLE)); assertEquals(mockClient.getAccessCount(), 1); metastore.flushCache(); - assertNotNull(metastore.getTable(SESSION.getUser(), TEST_DATABASE, TEST_TABLE)); + assertNotNull(metastore.getTable(TEST_DATABASE, TEST_TABLE)); assertEquals(mockClient.getAccessCount(), 2); } public void testInvalidDbGetTable() throws Exception { - assertFalse(metastore.getTable(SESSION.getUser(), BAD_DATABASE, TEST_TABLE).isPresent()); + assertFalse(metastore.getTable(BAD_DATABASE, TEST_TABLE).isPresent()); } @Test @@ -117,14 +116,14 @@ public void testGetPartitionNames() { ImmutableList expectedPartitions = ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2); assertEquals(mockClient.getAccessCount(), 0); - assertEquals(metastore.getPartitionNames(SESSION.getUser(), TEST_DATABASE, TEST_TABLE).get(), expectedPartitions); + assertEquals(metastore.getPartitionNames(TEST_DATABASE, TEST_TABLE).get(), expectedPartitions); assertEquals(mockClient.getAccessCount(), 1); - assertEquals(metastore.getPartitionNames(SESSION.getUser(), TEST_DATABASE, TEST_TABLE).get(), expectedPartitions); + assertEquals(metastore.getPartitionNames(TEST_DATABASE, TEST_TABLE).get(), expectedPartitions); assertEquals(mockClient.getAccessCount(), 1); metastore.flushCache(); - assertEquals(metastore.getPartitionNames(SESSION.getUser(), TEST_DATABASE, TEST_TABLE).get(), expectedPartitions); + assertEquals(metastore.getPartitionNames(TEST_DATABASE, TEST_TABLE).get(), expectedPartitions); assertEquals(mockClient.getAccessCount(), 2); } @@ -132,7 +131,7 @@ public void testGetPartitionNames() public void testInvalidGetPartitionNames() throws Exception { - assertEquals(metastore.getPartitionNames(SESSION.getUser(), BAD_DATABASE, TEST_TABLE).get(), ImmutableList.of()); + assertEquals(metastore.getPartitionNames(BAD_DATABASE, TEST_TABLE).get(), ImmutableList.of()); } @Test @@ -143,14 +142,14 @@ public void testGetPartitionNamesByParts() ImmutableList expectedPartitions = ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2); assertEquals(mockClient.getAccessCount(), 0); - assertEquals(metastore.getPartitionNamesByParts(SESSION.getUser(), TEST_DATABASE, TEST_TABLE, parts).get(), expectedPartitions); + assertEquals(metastore.getPartitionNamesByParts(TEST_DATABASE, TEST_TABLE, parts).get(), expectedPartitions); assertEquals(mockClient.getAccessCount(), 1); - assertEquals(metastore.getPartitionNamesByParts(SESSION.getUser(), TEST_DATABASE, TEST_TABLE, parts).get(), expectedPartitions); + assertEquals(metastore.getPartitionNamesByParts(TEST_DATABASE, TEST_TABLE, parts).get(), expectedPartitions); assertEquals(mockClient.getAccessCount(), 1); metastore.flushCache(); - assertEquals(metastore.getPartitionNamesByParts(SESSION.getUser(), TEST_DATABASE, TEST_TABLE, parts).get(), expectedPartitions); + assertEquals(metastore.getPartitionNamesByParts(TEST_DATABASE, TEST_TABLE, parts).get(), expectedPartitions); assertEquals(mockClient.getAccessCount(), 2); } @@ -158,7 +157,7 @@ public void testInvalidGetPartitionNamesByParts() throws Exception { ImmutableList parts = ImmutableList.of(); - assertFalse(metastore.getPartitionNamesByParts(SESSION.getUser(), BAD_DATABASE, TEST_TABLE, parts).isPresent()); + assertFalse(metastore.getPartitionNamesByParts(BAD_DATABASE, TEST_TABLE, parts).isPresent()); } @Test @@ -166,35 +165,35 @@ public void testGetPartitionsByNames() throws Exception { assertEquals(mockClient.getAccessCount(), 0); - metastore.getTable(SESSION.getUser(), TEST_DATABASE, TEST_TABLE); + metastore.getTable(TEST_DATABASE, TEST_TABLE); assertEquals(mockClient.getAccessCount(), 1); // Select half of the available partitions and load them into the cache - assertEquals(metastore.getPartitionsByNames(SESSION.getUser(), TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1)).get().size(), 1); + assertEquals(metastore.getPartitionsByNames(TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1)).get().size(), 1); assertEquals(mockClient.getAccessCount(), 2); // Now select all of the partitions - assertEquals(metastore.getPartitionsByNames(SESSION.getUser(), TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2)).get().size(), 2); + assertEquals(metastore.getPartitionsByNames(TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2)).get().size(), 2); // There should be one more access to fetch the remaining partition assertEquals(mockClient.getAccessCount(), 3); // Now if we fetch any or both of them, they should not hit the client - assertEquals(metastore.getPartitionsByNames(SESSION.getUser(), TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1)).get().size(), 1); - assertEquals(metastore.getPartitionsByNames(SESSION.getUser(), TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION2)).get().size(), 1); - assertEquals(metastore.getPartitionsByNames(SESSION.getUser(), TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2)).get().size(), 2); + assertEquals(metastore.getPartitionsByNames(TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1)).get().size(), 1); + assertEquals(metastore.getPartitionsByNames(TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION2)).get().size(), 1); + assertEquals(metastore.getPartitionsByNames(TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2)).get().size(), 2); assertEquals(mockClient.getAccessCount(), 3); metastore.flushCache(); // Fetching both should only result in one batched access - assertEquals(metastore.getPartitionsByNames(SESSION.getUser(), TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2)).get().size(), 2); + assertEquals(metastore.getPartitionsByNames(TEST_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1, TEST_PARTITION2)).get().size(), 2); assertEquals(mockClient.getAccessCount(), 4); } public void testInvalidGetPartitionsByNames() throws Exception { - assertFalse(metastore.getPartitionsByNames(SESSION.getUser(), BAD_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1)).isPresent()); + assertFalse(metastore.getPartitionsByNames(BAD_DATABASE, TEST_TABLE, ImmutableList.of(TEST_PARTITION1)).isPresent()); } @Test @@ -204,7 +203,7 @@ public void testNoCacheExceptions() // Throw exceptions on usage mockClient.setThrowException(true); try { - metastore.getAllDatabases(SESSION.getUser()); + metastore.getAllDatabases(); } catch (RuntimeException ignored) { } @@ -212,7 +211,7 @@ public void testNoCacheExceptions() // Second try should hit the client again try { - metastore.getAllDatabases(SESSION.getUser()); + metastore.getAllDatabases(); } catch (RuntimeException ignored) { } From 8c99eebea149686c123cd1a160d2e8bea9a00ca9 Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Mon, 23 Nov 2015 10:54:04 -0800 Subject: [PATCH 021/331] Revert "Load splits as user so that hdfs reads happen as that user." This reverts commit a39315881019902b237d11540219a1c94b7ecc0a. --- .../com/facebook/presto/hive/BackgroundHiveSplitLoader.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) 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 557544a96b6b..82defdecc369 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 @@ -38,10 +38,8 @@ import org.apache.hadoop.mapred.InputFormat; import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.security.UserGroupInformation; import java.io.IOException; -import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.Collections; import java.util.Deque; @@ -164,9 +162,8 @@ public TaskStatus process() try { CompletableFuture future; taskExecutionLock.readLock().lock(); - UserGroupInformation ugi = UserGroupInformation.createRemoteUser(session.getUser()); try { - future = ugi.doAs((PrivilegedExceptionAction>) BackgroundHiveSplitLoader.this::loadSplits); + future = loadSplits(); } finally { taskExecutionLock.readLock().unlock(); From 884902f5dfd41f6be39454bc1c90afee9d4a3f3b Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Mon, 23 Nov 2015 10:54:41 -0800 Subject: [PATCH 022/331] Revert "UserBasedHiveMetastore to access metastore as user issuing the query." This reverts commit 1b65e2fad9ac91da587e0193b114933461b7fcb6. --- .../presto/hive/HiveClientModule.java | 6 +- .../hive/ThriftHiveMetastoreClient.java | 5 - .../metastore/UserBasedHiveMetastore.java | 608 ------------------ .../UserBasedHiveMetastoreStats.java | 141 ---- 4 files changed, 3 insertions(+), 757 deletions(-) delete mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/metastore/UserBasedHiveMetastore.java delete mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/metastore/UserBasedHiveMetastoreStats.java 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 de95fec6e6fb..bc48cbfc7f0f 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 @@ -13,8 +13,8 @@ */ package com.facebook.presto.hive; +import com.facebook.presto.hive.metastore.CachingHiveMetastore; import com.facebook.presto.hive.metastore.HiveMetastore; -import com.facebook.presto.hive.metastore.UserBasedHiveMetastore; import com.facebook.presto.hive.orc.DwrfPageSourceFactory; import com.facebook.presto.hive.orc.DwrfRecordCursorProvider; import com.facebook.presto.hive.orc.OrcPageSourceFactory; @@ -81,9 +81,9 @@ public void configure(Binder binder) binder.bind(HiveMetastore.class).toInstance(metastore); } else { - binder.bind(HiveMetastore.class).to(UserBasedHiveMetastore.class).in(Scopes.SINGLETON); + binder.bind(HiveMetastore.class).to(CachingHiveMetastore.class).in(Scopes.SINGLETON); newExporter(binder).export(HiveMetastore.class) - .as(generatedNameOf(UserBasedHiveMetastore.class, connectorId)); + .as(generatedNameOf(CachingHiveMetastore.class, connectorId)); } binder.bind(NamenodeStats.class).in(Scopes.SINGLETON); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveMetastoreClient.java b/presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveMetastoreClient.java index 9ab9986b7bab..d97aaf3d464f 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveMetastoreClient.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveMetastoreClient.java @@ -49,11 +49,6 @@ public ThriftHiveMetastoreClient(TProtocol protocol) this.client = new ThriftHiveMetastore.Client(protocol); } - public ThriftHiveMetastore.Client getClient() - { - return client; - } - @Override public void close() { diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/UserBasedHiveMetastore.java b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/UserBasedHiveMetastore.java deleted file mode 100644 index 251962a01cc9..000000000000 --- a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/UserBasedHiveMetastore.java +++ /dev/null @@ -1,608 +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.metastore; - -import com.facebook.presto.hive.HiveCluster; -import com.facebook.presto.hive.HiveViewNotSupportedException; -import com.facebook.presto.hive.TableAlreadyExistsException; -import com.facebook.presto.hive.ThriftHiveMetastoreClient; -import com.facebook.presto.spi.PrestoException; -import com.facebook.presto.spi.SchemaNotFoundException; -import com.facebook.presto.spi.SchemaTableName; -import com.facebook.presto.spi.TableNotFoundException; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; -import org.apache.hadoop.hive.common.FileUtils; -import org.apache.hadoop.hive.metastore.TableType; -import org.apache.hadoop.hive.metastore.Warehouse; -import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; -import org.apache.hadoop.hive.metastore.api.Database; -import org.apache.hadoop.hive.metastore.api.HiveObjectRef; -import org.apache.hadoop.hive.metastore.api.HiveObjectType; -import org.apache.hadoop.hive.metastore.api.InvalidObjectException; -import org.apache.hadoop.hive.metastore.api.InvalidOperationException; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; -import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet; -import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo; -import org.apache.hadoop.hive.metastore.api.Role; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.api.UnknownDBException; -import org.apache.thrift.TException; -import org.weakref.jmx.Flatten; -import org.weakref.jmx.Managed; - -import javax.inject.Inject; - -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.Callable; -import java.util.function.Function; - -import static com.facebook.presto.hive.HiveErrorCode.HIVE_METASTORE_ERROR; -import static com.facebook.presto.hive.HiveUtil.PRESTO_VIEW_FLAG; -import static com.facebook.presto.hive.HiveUtil.isPrestoView; -import static com.facebook.presto.hive.RetryDriver.retry; -import static com.facebook.presto.hive.metastore.HivePrivilege.OWNERSHIP; -import static com.facebook.presto.hive.metastore.HivePrivilege.parsePrivilege; -import static com.google.common.base.Preconditions.checkArgument; -import static java.lang.String.format; -import static java.util.Objects.requireNonNull; -import static java.util.stream.Collectors.toSet; -import static org.apache.hadoop.hive.metastore.api.PrincipalType.USER; -import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.HIVE_FILTER_FIELD_PARAMS; - -public class UserBasedHiveMetastore - implements HiveMetastore -{ - private final UserBasedHiveMetastoreStats stats = new UserBasedHiveMetastoreStats(); - protected final HiveCluster clientProvider; - - @Inject - public UserBasedHiveMetastore(HiveCluster hiveCluster) - { - this.clientProvider = requireNonNull(hiveCluster, "hiveCluster is null"); - } - - @Managed - @Flatten - public UserBasedHiveMetastoreStats getStats() - { - return stats; - } - - protected Function getExceptionMapper() - { - return Function.identity(); - } - - @Override - public void createTable(String user, Table table) - { - try { - retry() - .exceptionMapper(getExceptionMapper()) - .stopOn(AlreadyExistsException.class, InvalidObjectException.class, MetaException.class, NoSuchObjectException.class) - .stopOnIllegalExceptions() - .run("createTable", stats.getCreateTable().wrap(() -> { - try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { - client.getClient().set_ugi(user, ImmutableList.of()); - client.createTable(table); - } - return null; - })); - } - catch (AlreadyExistsException e) { - throw new TableAlreadyExistsException(new SchemaTableName(table.getDbName(), table.getTableName())); - } - catch (NoSuchObjectException e) { - throw new SchemaNotFoundException(table.getDbName()); - } - catch (TException e) { - throw new PrestoException(HIVE_METASTORE_ERROR, e); - } - catch (Exception e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw Throwables.propagate(e); - } - } - - @Override - public void dropTable(String user, String databaseName, String tableName) - { - try { - retry() - .stopOn(org.apache.hadoop.hive.metastore.api.NoSuchObjectException.class) - .stopOnIllegalExceptions() - .run("dropTable", stats.getDropTable().wrap(() -> { - try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { - client.getClient().set_ugi(user, ImmutableList.of()); - client.dropTable(databaseName, tableName, true); - } - return null; - })); - } - catch (org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { - throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); - } - catch (TException e) { - throw new PrestoException(HIVE_METASTORE_ERROR, e); - } - catch (Exception e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw com.google.common.base.Throwables.propagate(e); - } - } - - @Override - public void alterTable(String user, String databaseName, String tableName, Table table) - { - try { - retry() - .exceptionMapper(getExceptionMapper()) - .stopOn(InvalidOperationException.class, MetaException.class) - .stopOnIllegalExceptions() - .run("alterTable", stats.getAlterTable().wrap(() -> { - try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { - Optional
source = getTable(user, databaseName, tableName); - if (!source.isPresent()) { - throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); - } - client.getClient().set_ugi(user, ImmutableList.of()); - client.alterTable(databaseName, tableName, table); - } - return null; - })); - } - catch (org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { - throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); - } - catch (InvalidOperationException | MetaException e) { - throw com.google.common.base.Throwables.propagate(e); - } - catch (TException e) { - throw new PrestoException(HIVE_METASTORE_ERROR, e); - } - catch (Exception e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw com.google.common.base.Throwables.propagate(e); - } - } - - @Override - public void flushCache() - { - } - - @Override - public List getAllDatabases(String user) - { - try { - return retry() - .stopOnIllegalExceptions() - .run("getAllDatabases", stats.getGetAllDatabases().wrap(() -> { - try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { - client.getClient().set_ugi(user, ImmutableList.of()); - return client.getAllDatabases(); - } - })); - } - catch (Exception e) { - throw new PrestoException(HIVE_METASTORE_ERROR, e); - } - } - - @Override - public Optional> getAllTables(String user, String databaseName) - { - Callable> getAllTables = stats.getGetAllTables().wrap(() -> { - try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { - client.getClient().set_ugi(user, ImmutableList.of()); - return client.getAllTables(databaseName); - } - }); - - Callable getDatabase = stats.getGetDatabase().wrap(() -> { - try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { - client.getClient().set_ugi(user, ImmutableList.of()); - client.getDatabase(databaseName); - return null; - } - }); - - try { - return retry() - .stopOn(org.apache.hadoop.hive.metastore.api.NoSuchObjectException.class) - .stopOnIllegalExceptions() - .run("getAllTables", () -> { - List tables = getAllTables.call(); - if (tables.isEmpty()) { - // Check to see if the database exists - getDatabase.call(); - } - return Optional.of(tables); - }); - } - catch (org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { - return Optional.empty(); - } - catch (Exception e) { - throw new PrestoException(HIVE_METASTORE_ERROR, e); - } - } - - @Override - public Optional> getAllViews(String user, String databaseName) - { - try { - return retry() - .stopOn(UnknownDBException.class) - .stopOnIllegalExceptions() - .run("getAllViews", stats.getAllViews().wrap(() -> { - try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { - String filter = HIVE_FILTER_FIELD_PARAMS + PRESTO_VIEW_FLAG + " = \"true\""; - client.getClient().set_ugi(user, ImmutableList.of()); - return Optional.of(client.getTableNamesByFilter(databaseName, filter)); - } - })); - } - catch (UnknownDBException e) { - return Optional.empty(); - } - catch (Exception e) { - throw new PrestoException(HIVE_METASTORE_ERROR, e); - } - } - - @Override - public Optional getDatabase(String user, String databaseName) - { - try { - return retry() - .stopOn(org.apache.hadoop.hive.metastore.api.NoSuchObjectException.class) - .stopOnIllegalExceptions() - .run("getDatabase", stats.getGetDatabase().wrap(() -> { - try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { - client.getClient().set_ugi(user, ImmutableList.of()); - return Optional.of(client.getDatabase(databaseName)); - } - })); - } - catch (org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { - return Optional.empty(); - } - catch (Exception e) { - throw new PrestoException(HIVE_METASTORE_ERROR, e); - } - } - - @Override - public void addPartitions(String user, String databaseName, String tableName, List partitions) - { - if (partitions.isEmpty()) { - return; - } - try { - retry() - .exceptionMapper(getExceptionMapper()) - .stopOn(org.apache.hadoop.hive.metastore.api.AlreadyExistsException.class, org.apache.hadoop.hive.metastore.api.InvalidObjectException.class, MetaException.class, org.apache.hadoop.hive.metastore.api.NoSuchObjectException.class, PrestoException.class) - .stopOnIllegalExceptions() - .run("addPartitions", stats.getAddPartitions().wrap(() -> { - try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { - client.getClient().set_ugi(user, ImmutableList.of()); - int partitionsAdded = client.addPartitions(partitions); - if (partitionsAdded != partitions.size()) { - throw new PrestoException(HIVE_METASTORE_ERROR, - format("Hive metastore only added %s of %s partitions", partitionsAdded, partitions.size())); - } - } - return null; - })); - } - catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException | org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { - // todo partition already exists exception - throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); - } - catch (TException e) { - throw new PrestoException(HIVE_METASTORE_ERROR, e); - } - catch (Exception e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw com.google.common.base.Throwables.propagate(e); - } - } - - @Override - public void dropPartition(String user, String databaseName, String tableName, List parts) - { - try { - retry() - .stopOn(org.apache.hadoop.hive.metastore.api.NoSuchObjectException.class, MetaException.class) - .stopOnIllegalExceptions() - .run("dropPartition", stats.getDropPartition().wrap(() -> { - try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { - client.getClient().set_ugi(user, ImmutableList.of()); - client.dropPartition(databaseName, tableName, parts, true); - } - return null; - })); - } - catch (org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { - throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); - } - catch (TException e) { - throw new PrestoException(HIVE_METASTORE_ERROR, e); - } - catch (Exception e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw com.google.common.base.Throwables.propagate(e); - } - } - - @Override - public void dropPartitionByName(String user, String databaseName, String tableName, String partitionName) - { - try { - retry() - .stopOn(org.apache.hadoop.hive.metastore.api.NoSuchObjectException.class, MetaException.class) - .stopOnIllegalExceptions() - .run("dropPartitionByName", stats.getDropPartitionByName().wrap(() -> { - try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { - // It is observed that: (examples below assumes a table with one partition column `ds`) - // * When a partition doesn't exist (e.g. ds=2015-09-99), this thrift call is a no-op. It doesn't throw any exception. - // * When a typo exists in partition column name (e.g. dxs=2015-09-01), this thrift call will delete ds=2015-09-01. - client.getClient().set_ugi(user, ImmutableList.of()); - client.dropPartitionByName(databaseName, tableName, partitionName, true); - } - return null; - })); - } - catch (org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { - throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); - } - catch (TException e) { - throw new PrestoException(HIVE_METASTORE_ERROR, e); - } - catch (Exception e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw com.google.common.base.Throwables.propagate(e); - } - } - - @Override - public Optional> getPartitionNames(String user, String databaseName, String tableName) - { - try { - return retry() - .stopOn(org.apache.hadoop.hive.metastore.api.NoSuchObjectException.class) - .stopOnIllegalExceptions() - .run("getPartitionNames", stats.getGetPartitionNames().wrap(() -> { - try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { - client.getClient().set_ugi(user, ImmutableList.of()); - return Optional.of(client.getPartitionNames(databaseName, tableName)); - } - })); - } - catch (org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { - return Optional.empty(); - } - catch (Exception e) { - throw new PrestoException(HIVE_METASTORE_ERROR, e); - } - } - - @Override - public Optional> getPartitionNamesByParts(String user, String databaseName, String tableName, List parts) - { - try { - return retry() - .stopOn(org.apache.hadoop.hive.metastore.api.NoSuchObjectException.class) - .stopOnIllegalExceptions() - .run("getPartitionNamesByParts", stats.getGetPartitionNamesPs().wrap(() -> { - try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { - client.getClient().set_ugi(user, ImmutableList.of()); - return Optional.of(client.getPartitionNamesFiltered(databaseName, tableName, parts)); - } - })); - } - catch (org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { - return Optional.empty(); - } - catch (Exception e) { - throw new PrestoException(HIVE_METASTORE_ERROR, e); - } - } - - @Override - public Optional getPartition(String user, String databaseName, String tableName, String partitionName) - { - requireNonNull(partitionName, "partitionName is null"); - try { - return retry() - .stopOn(org.apache.hadoop.hive.metastore.api.NoSuchObjectException.class) - .stopOnIllegalExceptions() - .run("getPartitionsByNames", stats.getGetPartitionByName().wrap(() -> { - try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { - client.getClient().set_ugi(user, ImmutableList.of()); - return Optional.of(client.getPartitionByName(databaseName, tableName, partitionName)); - } - })); - } - catch (org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { - return Optional.empty(); - } - catch (Exception e) { - throw new PrestoException(HIVE_METASTORE_ERROR, e); - } - } - - @Override - public Optional> getPartitionsByNames(String user, String databaseName, String tableName, List partitionNames) - { - requireNonNull(partitionNames, "partitionNames is null"); - checkArgument(!Iterables.isEmpty(partitionNames), "partitionNames is empty"); - - try { - return retry() - .stopOn(org.apache.hadoop.hive.metastore.api.NoSuchObjectException.class) - .stopOnIllegalExceptions() - .run("getPartitionsByNames", stats.getGetPartitionsByNames().wrap(() -> { - try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { - List partitionColumnNames = ImmutableList.copyOf(Warehouse.makeSpecFromName(partitionNames.get(0)).keySet()); - ImmutableMap.Builder partitions = ImmutableMap.builder(); - client.getClient().set_ugi(user, ImmutableList.of()); - for (Partition partition : client.getPartitionsByNames(databaseName, tableName, partitionNames)) { - String partitionId = FileUtils.makePartName(partitionColumnNames, partition.getValues(), null); - partitions.put(partitionId, partition); - } - return Optional.of(partitions.build()); - } - })); - } - catch (org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { - // assume none of the partitions in the batch are available - return Optional.empty(); - } - catch (Exception e) { - throw new PrestoException(HIVE_METASTORE_ERROR, e); - } - } - - @Override - public Optional
getTable(String user, String databaseName, String tableName) - { - try { - return retry() - .stopOn(org.apache.hadoop.hive.metastore.api.NoSuchObjectException.class, HiveViewNotSupportedException.class) - .stopOnIllegalExceptions() - .run("getTable", stats.getGetTable().wrap(() -> { - try (ThriftHiveMetastoreClient client = (ThriftHiveMetastoreClient) clientProvider.createMetastoreClient()) { - client.getClient().set_ugi(user, ImmutableList.of()); - Table table = client.getTable(databaseName, tableName); - if (table.getTableType().equals(TableType.VIRTUAL_VIEW.name()) && (!isPrestoView(table))) { - throw new HiveViewNotSupportedException(new SchemaTableName(databaseName, tableName)); - } - return Optional.of(table); - } - })); - } - catch (org.apache.hadoop.hive.metastore.api.NoSuchObjectException e) { - return Optional.empty(); - } - catch (Exception e) { - throw new PrestoException(HIVE_METASTORE_ERROR, e); - } - } - - @Override - public Set getRoles(String user) - { - try (HiveMetastoreClient client = clientProvider.createMetastoreClient()) { - List roles = client.listRoles(user, USER); - if (roles == null) { - return ImmutableSet.of(); - } - return ImmutableSet.copyOf(roles.stream() - .map(Role::getRoleName) - .collect(toSet())); - } - catch (TException e) { - throw new PrestoException(HIVE_METASTORE_ERROR, e); - } - } - - @Override - public Set getDatabasePrivileges(String user, String databaseName) - { - ImmutableSet.Builder privileges = ImmutableSet.builder(); - - if (isDatabaseOwner(user, databaseName)) { - privileges.add(OWNERSHIP); - } - privileges.addAll(getPrivileges(user, new HiveObjectRef(HiveObjectType.DATABASE, databaseName, null, null, null))); - - return privileges.build(); - } - - @Override - public Set getTablePrivileges(String user, String databaseName, String tableName) - { - ImmutableSet.Builder privileges = ImmutableSet.builder(); - - if (isTableOwner(user, databaseName, tableName)) { - privileges.add(OWNERSHIP); - } - privileges.addAll(getPrivileges(user, new HiveObjectRef(HiveObjectType.TABLE, databaseName, tableName, null, null))); - - return privileges.build(); - } - - private Set getPrivileges(String user, HiveObjectRef objectReference) - { - ImmutableSet.Builder privileges = ImmutableSet.builder(); - try (HiveMetastoreClient client = clientProvider.createMetastoreClient()) { - PrincipalPrivilegeSet privilegeSet = client.getPrivilegeSet(objectReference, user, null); - - if (privilegeSet != null) { - Map> userPrivileges = privilegeSet.getUserPrivileges(); - if (userPrivileges != null) { - privileges.addAll(toGrants(userPrivileges.get(user))); - } - for (List rolePrivileges : privilegeSet.getRolePrivileges().values()) { - privileges.addAll(toGrants(rolePrivileges)); - } - // We do not add the group permissions as Hive does not seem to process these - } - } - catch (TException e) { - throw new PrestoException(HIVE_METASTORE_ERROR, e); - } - - return privileges.build(); - } - - private static Set toGrants(List userGrants) - { - if (userGrants == null) { - return ImmutableSet.of(); - } - - ImmutableSet.Builder privileges = ImmutableSet.builder(); - for (PrivilegeGrantInfo userGrant : userGrants) { - privileges.addAll(parsePrivilege(userGrant)); - if (userGrant.isGrantOption()) { - privileges.add(HivePrivilege.GRANT); - } - } - return privileges.build(); - } -} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/UserBasedHiveMetastoreStats.java b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/UserBasedHiveMetastoreStats.java deleted file mode 100644 index 7778e2ba9010..000000000000 --- a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/UserBasedHiveMetastoreStats.java +++ /dev/null @@ -1,141 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.facebook.presto.hive.metastore; - -import org.weakref.jmx.Managed; -import org.weakref.jmx.Nested; - -public class UserBasedHiveMetastoreStats -{ - private final HiveMetastoreApiStats getAllDatabases = new HiveMetastoreApiStats(); - private final HiveMetastoreApiStats getDatabase = new HiveMetastoreApiStats(); - private final HiveMetastoreApiStats getAllTables = new HiveMetastoreApiStats(); - private final HiveMetastoreApiStats getAllViews = new HiveMetastoreApiStats(); - private final HiveMetastoreApiStats getTable = new HiveMetastoreApiStats(); - private final HiveMetastoreApiStats getPartitionNames = new HiveMetastoreApiStats(); - private final HiveMetastoreApiStats getPartitionNamesPs = new HiveMetastoreApiStats(); - private final HiveMetastoreApiStats getPartitionByName = new HiveMetastoreApiStats(); - private final HiveMetastoreApiStats getPartitionsByNames = new HiveMetastoreApiStats(); - private final HiveMetastoreApiStats createTable = new HiveMetastoreApiStats(); - private final HiveMetastoreApiStats dropTable = new HiveMetastoreApiStats(); - private final HiveMetastoreApiStats alterTable = new HiveMetastoreApiStats(); - private final HiveMetastoreApiStats addPartitions = new HiveMetastoreApiStats(); - private final HiveMetastoreApiStats dropPartition = new HiveMetastoreApiStats(); - private final HiveMetastoreApiStats dropPartitionByName = new HiveMetastoreApiStats(); - - @Managed - @Nested - public HiveMetastoreApiStats getGetAllDatabases() - { - return getAllDatabases; - } - - @Managed - @Nested - public HiveMetastoreApiStats getGetDatabase() - { - return getDatabase; - } - - @Managed - @Nested - public HiveMetastoreApiStats getGetAllTables() - { - return getAllTables; - } - - @Managed - @Nested - public HiveMetastoreApiStats getAllViews() - { - return getAllViews; - } - - @Managed - @Nested - public HiveMetastoreApiStats getGetTable() - { - return getTable; - } - - @Managed - @Nested - public HiveMetastoreApiStats getGetPartitionNames() - { - return getPartitionNames; - } - - @Managed - @Nested - public HiveMetastoreApiStats getGetPartitionNamesPs() - { - return getPartitionNamesPs; - } - - @Managed - @Nested - public HiveMetastoreApiStats getGetPartitionByName() - { - return getPartitionByName; - } - - @Managed - @Nested - public HiveMetastoreApiStats getGetPartitionsByNames() - { - return getPartitionsByNames; - } - - @Managed - @Nested - public HiveMetastoreApiStats getCreateTable() - { - return createTable; - } - - @Managed - @Nested - public HiveMetastoreApiStats getDropTable() - { - return dropTable; - } - - @Managed - @Nested - public HiveMetastoreApiStats getAlterTable() - { - return alterTable; - } - - @Managed - @Nested - public HiveMetastoreApiStats getAddPartitions() - { - return addPartitions; - } - - @Managed - @Nested - public HiveMetastoreApiStats getDropPartition() - { - return dropPartition; - } - - @Managed - @Nested - public HiveMetastoreApiStats getDropPartitionByName() - { - return dropPartitionByName; - } -} From fe37227c083ee62b209845b90799d4aa0ea3a238 Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Mon, 23 Nov 2015 11:45:41 -0800 Subject: [PATCH 023/331] Prepare for release 0.126-tw-0.7. --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 36 files changed, 37 insertions(+), 37 deletions(-) diff --git a/pom.xml b/pom.xml index 52e309f6df08..414fc84a115a 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.126 + 0.126-tw-0.7 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 0cc6cf09234b..9a133d8e8f0f 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 9e0761b15f3d..3f218464e80f 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 3866f58c52ee..96fc271fb37c 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.126 + 0.126-tw-0.7 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index e91a54b509c9..5344477eda37 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 03ef5b2a0230..0a882093d15c 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 46f0657d2713..ad4ec98b26bc 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 93a19ec2c54b..ac9647ce93ab 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 55dc44d383e4..f675e09c07fa 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 52f83773c3fd..e459541f3f49 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index a5a2688113f8..3a153832ad5b 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 38793b1cc542..4b412e4133e1 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 7fcd9c617283..c1bfac684b13 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 0c6c17a88228..29994c4839ac 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 9a80e1608146..5498b04ad31e 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index d1b20af9b4b8..275b9946f55d 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 5934d498d5f8..5c7cd51fbd15 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 0ed7ee924f7d..bac1e1f514d0 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index ea9e6d448a2a..013447fe2411 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index c0c6cf3bdba7..ddaf72fbbe69 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 2d4c8893a501..45c9d8a3ee59 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 224f07d32d5b..892e52942c88 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index baa497e03aa1..a04a1810cacf 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 31111245dbea..7e2c6e0aafe6 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index ada21024306c..d1056e3df9a7 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.126 + 0.126-tw-0.7 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 71f1ea8d79e3..d9a3f6ef1fbb 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 7fcfc62e80c6..2bfe3c924b40 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index e438bee2d5a7..e512dc5d32f8 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index d6dc67beb915..e053c3aac08b 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index e919eea24cdf..5832a24c82d5 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index df3aaba4e79f..5c9bf5584c84 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index b069db9c4c96..a9344873b22f 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 59fbc9d4518f..966add9b8b0a 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 5842d70b3388..016b69ffc3b1 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.126 + 0.126-tw-0.7 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 8362c3721e8a..60821655b780 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index cc46b69468d1..08dafd617da6 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126 + 0.126-tw-0.7 presto-verifier From 344c8f0b4e90e38a26f443b9ad090f632ae54478 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Tue, 8 Dec 2015 11:19:49 -0800 Subject: [PATCH 024/331] HiveSplitManager to start HiveSplitLoader as session user --- .../presto/hive/HiveSplitManager.java | 20 +++++++++++++++++-- 1 file changed, 18 insertions(+), 2 deletions(-) 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 8f63692545bc..fcca3ab5bdc1 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 @@ -34,9 +34,12 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.security.UserGroupInformation; import javax.inject.Inject; +import java.io.IOException; +import java.security.PrivilegedExceptionAction; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -180,9 +183,22 @@ public ConnectorSplitSource getSplits(ConnectorSession session, ConnectorTableLa recursiveDfsWalkerEnabled); HiveSplitSource splitSource = new HiveSplitSource(connectorId, maxOutstandingSplits, hiveSplitLoader, executor); - hiveSplitLoader.start(splitSource); - return splitSource; + UserGroupInformation ugi = UserGroupInformation.createRemoteUser(session.getUser()); + + try { + ugi.doAs((PrivilegedExceptionAction) + () -> { + hiveSplitLoader.start(splitSource); + return null; + } + ); + } + catch (IOException | InterruptedException e) { + throw new RuntimeException("Could not runAs " + ugi.getUserName(), e); + } + + return splitSource; } private Iterable getPartitionMetadata(Table table, SchemaTableName tableName, List hivePartitions) From 378c4ed24ff10ca8954d1857aa1fd55d949c7873 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Tue, 8 Dec 2015 12:30:36 -0800 Subject: [PATCH 025/331] fixing indentation --- .../facebook/presto/hive/HiveSplitManager.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) 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 fcca3ab5bdc1..b04d13ae99cf 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 @@ -187,18 +187,18 @@ public ConnectorSplitSource getSplits(ConnectorSession session, ConnectorTableLa UserGroupInformation ugi = UserGroupInformation.createRemoteUser(session.getUser()); try { - ugi.doAs((PrivilegedExceptionAction) - () -> { - hiveSplitLoader.start(splitSource); - return null; - } - ); + ugi.doAs((PrivilegedExceptionAction) + () -> { + hiveSplitLoader.start(splitSource); + return null; + } + ); } catch (IOException | InterruptedException e) { - throw new RuntimeException("Could not runAs " + ugi.getUserName(), e); + throw new RuntimeException("Could not runAs " + ugi.getUserName(), e); } - return splitSource; + return splitSource; } private Iterable getPartitionMetadata(Table table, SchemaTableName tableName, List hivePartitions) From 484e2554ecd3f8695231f2ca624a7578c33d6686 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Tue, 8 Dec 2015 12:48:49 -0800 Subject: [PATCH 026/331] fixing indentation --- .../java/com/facebook/presto/hive/HiveSplitManager.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 b04d13ae99cf..aa6ba715b7f6 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 @@ -188,10 +188,10 @@ public ConnectorSplitSource getSplits(ConnectorSession session, ConnectorTableLa try { ugi.doAs((PrivilegedExceptionAction) - () -> { - hiveSplitLoader.start(splitSource); - return null; - } + () -> { + hiveSplitLoader.start(splitSource); + return null; + } ); } catch (IOException | InterruptedException e) { From 4300a90172c84b8f5cd3587d1c0a4358b5114e40 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Tue, 8 Dec 2015 21:17:06 -0800 Subject: [PATCH 027/331] Updating version from to 0.126-tw-0.8 --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 36 files changed, 37 insertions(+), 37 deletions(-) diff --git a/pom.xml b/pom.xml index 414fc84a115a..87c608a57ce1 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.126-tw-0.7 + 0.126-tw-0.8 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 9a133d8e8f0f..2b54b534ec61 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 3f218464e80f..24f3f314d8cf 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 96fc271fb37c..60b0e963643a 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.126-tw-0.7 + 0.126-tw-0.8 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 5344477eda37..44d712cb2f74 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 0a882093d15c..bda12d19948e 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index ad4ec98b26bc..995e766931ac 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index ac9647ce93ab..235a29672d56 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index f675e09c07fa..f890cc393d9c 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index e459541f3f49..e3f6b094975c 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 3a153832ad5b..19a597ed590d 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 4b412e4133e1..1b00d6fbbc66 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index c1bfac684b13..e6884f7c1c4e 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 29994c4839ac..3cec99c7cbf4 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 5498b04ad31e..e837f4b0c0d6 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 275b9946f55d..38847ba75baa 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 5c7cd51fbd15..f7776398f801 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index bac1e1f514d0..a99b08424438 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 013447fe2411..195758c2b1ed 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index ddaf72fbbe69..1645784011d6 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 45c9d8a3ee59..3fdeff83cdaa 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 892e52942c88..85d19916b955 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index a04a1810cacf..be183abcd5ec 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 7e2c6e0aafe6..7f27364dea63 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index d1056e3df9a7..6b82404a6e8b 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.126-tw-0.7 + 0.126-tw-0.8 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index d9a3f6ef1fbb..e7fbecebc5bc 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 2bfe3c924b40..a47ea27e5f17 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index e512dc5d32f8..0493d38e3e4b 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index e053c3aac08b..67c18607589f 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 5832a24c82d5..cd423b82ff29 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 5c9bf5584c84..93a4a5070b16 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index a9344873b22f..b7542fc5167d 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 966add9b8b0a..969316f4e454 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 016b69ffc3b1..d1f9516597a3 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.126-tw-0.7 + 0.126-tw-0.8 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 60821655b780..4bfbd1272ef5 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 08dafd617da6..80e27a5c2df0 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.7 + 0.126-tw-0.8 presto-verifier From b00c53a3a350b295bdddaaabbaf4dd9806045cbf Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Wed, 16 Dec 2015 21:12:53 -0800 Subject: [PATCH 028/331] Revert runAsUser in preparation for 0.130 upgrade --- .../facebook/presto/hive/HiveSplitManager.java | 18 +----------------- 1 file changed, 1 insertion(+), 17 deletions(-) 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 aa6ba715b7f6..8f63692545bc 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 @@ -34,12 +34,9 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.security.UserGroupInformation; import javax.inject.Inject; -import java.io.IOException; -import java.security.PrivilegedExceptionAction; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -183,20 +180,7 @@ public ConnectorSplitSource getSplits(ConnectorSession session, ConnectorTableLa recursiveDfsWalkerEnabled); HiveSplitSource splitSource = new HiveSplitSource(connectorId, maxOutstandingSplits, hiveSplitLoader, executor); - - UserGroupInformation ugi = UserGroupInformation.createRemoteUser(session.getUser()); - - try { - ugi.doAs((PrivilegedExceptionAction) - () -> { - hiveSplitLoader.start(splitSource); - return null; - } - ); - } - catch (IOException | InterruptedException e) { - throw new RuntimeException("Could not runAs " + ugi.getUserName(), e); - } + hiveSplitLoader.start(splitSource); return splitSource; } From adb149c1a125ef6ddc7c97dc24084727b2396766 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Wed, 16 Dec 2015 21:29:15 -0800 Subject: [PATCH 029/331] Changing pom version to 0.130 to avoid merge conflicts when merging in 0.130 --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 36 files changed, 37 insertions(+), 37 deletions(-) diff --git a/pom.xml b/pom.xml index 87c608a57ce1..7b7b6132193d 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.126-tw-0.8 + 0.130 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 2b54b534ec61..2238de516139 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 24f3f314d8cf..c9055bbaf619 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 60b0e963643a..1503671c800f 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.126-tw-0.8 + 0.130 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 44d712cb2f74..a4db98340a48 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index bda12d19948e..e91e1350506c 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 995e766931ac..b1478999daae 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 235a29672d56..68904e2d5b5f 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index f890cc393d9c..a3fea8108719 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index e3f6b094975c..3c9c1d32c05c 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 19a597ed590d..fd26c464678e 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 1b00d6fbbc66..1ba9d816d070 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index e6884f7c1c4e..146eee7554fc 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 3cec99c7cbf4..476ff742e5fe 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index e837f4b0c0d6..c36826cb6065 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 38847ba75baa..511b51e67e91 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index f7776398f801..5781bb8854fe 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index a99b08424438..88092f60e0e7 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 195758c2b1ed..8c3f7eaa4550 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 1645784011d6..46f9c3b44552 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 3fdeff83cdaa..1f284d530c36 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 85d19916b955..e59590ac04ef 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index be183abcd5ec..7ebf10590ad7 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 7f27364dea63..23733201a738 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 6b82404a6e8b..d4bcbca7ca86 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.126-tw-0.8 + 0.130 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index e7fbecebc5bc..dfabac002bf5 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index a47ea27e5f17..f8e944107093 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 0493d38e3e4b..91b8272d48a6 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 67c18607589f..8747490178eb 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index cd423b82ff29..d2cf4984cc61 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 93a4a5070b16..aaf670e2913d 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index b7542fc5167d..c11cac6fa8a6 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 969316f4e454..986d71502f89 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index d1f9516597a3..624160ecc0c8 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.126-tw-0.8 + 0.130 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 4bfbd1272ef5..3be286cebe3a 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 80e27a5c2df0..6963319d3361 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.126-tw-0.8 + 0.130 presto-verifier From e84c4a4e50a5dd1555c4dcdde2531313429fe713 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Wed, 16 Dec 2015 21:36:07 -0800 Subject: [PATCH 030/331] Prepare for release 0.130-tw-0.9 --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 36 files changed, 37 insertions(+), 37 deletions(-) diff --git a/pom.xml b/pom.xml index 53aa0c6deed1..c627fc780d5a 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.130 + 0.130-tw-0.9 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 2238de516139..b1cdc92b7133 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index c9055bbaf619..2c1cf1e8232d 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 1503671c800f..71ac5b6b6a71 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.130 + 0.130-tw-0.9 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index a4db98340a48..7edb8a4b301f 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index e91e1350506c..70adcfa34e45 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index b1478999daae..9dcd4c01479e 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 68904e2d5b5f..7cee57559957 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index a3fea8108719..0a2a80999767 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 3c9c1d32c05c..c794568a3064 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index fd26c464678e..31a0e5011eba 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 1ba9d816d070..46480d8d82e9 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 146eee7554fc..64deba3e5bc7 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 476ff742e5fe..134384e326a8 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 71d9f68ff8b8..8bcbb3075b9d 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index d2555a1ddba6..c7d937faa290 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 5781bb8854fe..baa24b1ef8de 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 88092f60e0e7..ff01a6845134 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 8c3f7eaa4550..bcf29c7aa29a 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 46f9c3b44552..4b195e0d7925 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 1f284d530c36..d8639669a4bc 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 223ecf28ad9f..073f5f3cafa3 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 7ebf10590ad7..bbfefe90b54c 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 23733201a738..8801d9be3117 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index d4bcbca7ca86..7ff1c69c918d 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.130 + 0.130-tw-0.9 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 0cf57ffa36e9..b95b1d27d1fb 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index f8e944107093..4d22fb8ad760 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 91b8272d48a6..4f95e5638e5d 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 8747490178eb..db15e14079e3 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index d2cf4984cc61..8c6a691ba085 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 81ceb2227166..618475375aaf 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index c11cac6fa8a6..cecc9e929c4f 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 986d71502f89..e3aa177c836c 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 624160ecc0c8..07ecc82a8563 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.130 + 0.130-tw-0.9 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 3be286cebe3a..2ce2e06261ff 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 9ffdd566bcd6..5464d09e2217 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130 + 0.130-tw-0.9 presto-verifier From 3861dc30f59e8db824d84931694ce413564b5cb4 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Wed, 16 Dec 2015 23:41:42 -0800 Subject: [PATCH 031/331] Remove empty line to fix stylecheck error --- .../com/facebook/presto/hive/TestZookeeperMetastoreMonitor.java | 1 - 1 file changed, 1 deletion(-) diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestZookeeperMetastoreMonitor.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestZookeeperMetastoreMonitor.java index 9ccf999edac4..6b82efa22f97 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestZookeeperMetastoreMonitor.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestZookeeperMetastoreMonitor.java @@ -77,7 +77,6 @@ public void destroy() zkMetastoreMonitor.close(); zkClient.close(); zkServer.close(); - } @BeforeTest From 7dd60e54e7d853ea16e3d7473b2af0097037349c Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Tue, 22 Dec 2015 17:10:01 -0800 Subject: [PATCH 032/331] Allow Hive queries to read as the querying user --- .../hive/BackgroundHiveSplitLoader.java | 24 +++++++++++++++++++ .../presto/hive/HiveClientConfig.java | 15 ++++++++++++ .../presto/hive/HiveSessionProperties.java | 11 +++++++++ 3 files changed, 50 insertions(+) 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 0c854f66abcd..d15374abcf75 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 @@ -38,8 +38,10 @@ import org.apache.hadoop.mapred.InputFormat; import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.security.UserGroupInformation; import java.io.IOException; +import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.Collections; import java.util.Deque; @@ -154,6 +156,28 @@ private class HiveSplitLoaderTask { @Override public TaskStatus process() + { + if (HiveSessionProperties.getReadAsQueryUser(session)) { + try { + // TODO: Configure hadoop to allow sqlsystem to impersonate all presto users and + // change to the approach below. + // See https://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/hadoop-common/Superusers.html + // UserGroupInformation ugi = UserGroupInformation.createProxyUser( + // session.getUser(), UserGroupInformation.getLoginUser()); + UserGroupInformation ugi = UserGroupInformation.createRemoteUser(session.getUser()); + + return ugi.doAs((PrivilegedExceptionAction) this::doProcess); + } + catch (IOException | InterruptedException e) { + throw new RuntimeException("Could not runAs " + session.getUser(), e); + } + } + else { + return doProcess(); + } + } + + private TaskStatus doProcess() { while (true) { if (stopped) { 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 21cf49ce0b87..8ddca5cd58ba 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 @@ -67,6 +67,8 @@ public class HiveClientConfig private boolean allowCorruptWritesForTesting; + private boolean readAsQueryUser = false; + private Duration metastoreCacheTtl = new Duration(1, TimeUnit.HOURS); private Duration metastoreRefreshInterval = new Duration(1, TimeUnit.SECONDS); private int maxMetastoreRefreshThreads = 100; @@ -291,6 +293,19 @@ public HiveClientConfig setAllowCorruptWritesForTesting(boolean allowCorruptWrit return this; } + public boolean getReadAsQueryUser() + { + return readAsQueryUser; + } + + @Config("hive.read-as-query-user") + @ConfigDescription("When querying hive read data as the user submitting the query instead of as the presto daemon user") + public HiveClientConfig setReadAsQueryUser(boolean readAsQueryUser) + { + this.readAsQueryUser = readAsQueryUser; + return this; + } + public boolean getAllowAddColumn() { return this.allowAddColumn; 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 37bf715c825f..6dd1bc9852a0 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 @@ -28,6 +28,7 @@ public final class HiveSessionProperties { private static final String FORCE_LOCAL_SCHEDULING = "force_local_scheduling"; + private static final String READ_AS_QUERY_USER = "read_as_query_user"; private static final String OPTIMIZED_READER_ENABLED = "optimized_reader_enabled"; private static final String ORC_MAX_MERGE_DISTANCE = "orc_max_merge_distance"; private static final String ORC_MAX_BUFFER_SIZE = "orc_max_buffer_size"; @@ -51,6 +52,11 @@ public HiveSessionProperties(HiveClientConfig config) "Enable optimized readers", config.isOptimizedReaderEnabled(), true), + booleanSessionProperty( + READ_AS_QUERY_USER, + "Query reads happen as the user submitting the query", + config.getReadAsQueryUser(), + true), dataSizeSessionProperty( ORC_MAX_MERGE_DISTANCE, "ORC: Maximum size of gap between two reads to merge into a single read", @@ -93,6 +99,11 @@ public static boolean isOptimizedReaderEnabled(ConnectorSession session) return session.getProperty(OPTIMIZED_READER_ENABLED, Boolean.class); } + public static boolean getReadAsQueryUser(ConnectorSession session) + { + return session.getProperty(READ_AS_QUERY_USER, Boolean.class); + } + public static boolean isParquetOptimizedReaderEnabled(ConnectorSession session) { return session.getProperty(PARQUET_OPTIMIZED_READER_ENABLED, Boolean.class); From efc25af93ecf4c79f60aa874dc20c8781e735f1e Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Wed, 23 Dec 2015 09:45:28 -0800 Subject: [PATCH 033/331] add links to jiras --- .../com/facebook/presto/hive/BackgroundHiveSplitLoader.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 d15374abcf75..1123e33a1f42 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 @@ -159,8 +159,8 @@ public TaskStatus process() { if (HiveSessionProperties.getReadAsQueryUser(session)) { try { - // TODO: Configure hadoop to allow sqlsystem to impersonate all presto users and - // change to the approach below. + // TODO: Configure hadoop to allow sqlsystem to impersonate all presto users (HADOOPINFRA-7081) + // and then change to the approach below (IQ-85). // See https://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/hadoop-common/Superusers.html // UserGroupInformation ugi = UserGroupInformation.createProxyUser( // session.getUser(), UserGroupInformation.getLoginUser()); From ef06e47416518d7478ab721e01cc5a59cbeb0f81 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Wed, 23 Dec 2015 15:16:45 -0800 Subject: [PATCH 034/331] Clean up comments and constant order --- .../presto/hive/BackgroundHiveSplitLoader.java | 4 ++-- .../facebook/presto/hive/HiveSessionProperties.java | 12 ++++++------ 2 files changed, 8 insertions(+), 8 deletions(-) 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 1123e33a1f42..82b6d7cb54da 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 @@ -159,8 +159,8 @@ public TaskStatus process() { if (HiveSessionProperties.getReadAsQueryUser(session)) { try { - // TODO: Configure hadoop to allow sqlsystem to impersonate all presto users (HADOOPINFRA-7081) - // and then change to the approach below (IQ-85). + // TODO: Configure hadoop to allow presto daemon user to impersonate all presto users + // (HADOOPINFRA-7081) and then change to the approach below (IQ-85). // See https://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/hadoop-common/Superusers.html // UserGroupInformation ugi = UserGroupInformation.createProxyUser( // session.getUser(), UserGroupInformation.getLoginUser()); 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 6dd1bc9852a0..ab1d4d2c59bf 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 @@ -28,13 +28,13 @@ public final class HiveSessionProperties { private static final String FORCE_LOCAL_SCHEDULING = "force_local_scheduling"; - private static final String READ_AS_QUERY_USER = "read_as_query_user"; private static final String OPTIMIZED_READER_ENABLED = "optimized_reader_enabled"; 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 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"; private final List> sessionProperties; @@ -99,11 +99,6 @@ public static boolean isOptimizedReaderEnabled(ConnectorSession session) return session.getProperty(OPTIMIZED_READER_ENABLED, Boolean.class); } - public static boolean getReadAsQueryUser(ConnectorSession session) - { - return session.getProperty(READ_AS_QUERY_USER, Boolean.class); - } - public static boolean isParquetOptimizedReaderEnabled(ConnectorSession session) { return session.getProperty(PARQUET_OPTIMIZED_READER_ENABLED, Boolean.class); @@ -129,6 +124,11 @@ public static boolean isParquetPredicatePushdownEnabled(ConnectorSession session return session.getProperty(PARQUET_PREDICATE_PUSHDOWN_ENABLED, Boolean.class); } + public static boolean getReadAsQueryUser(ConnectorSession session) + { + return session.getProperty(READ_AS_QUERY_USER, Boolean.class); + } + public static PropertyMetadata dataSizeSessionProperty(String name, String description, DataSize defaultValue, boolean hidden) { return new PropertyMetadata<>( From 2200cdef2bbed2e6260dfd2d2abdaccc9bab3f80 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Mon, 4 Jan 2016 10:52:56 -0800 Subject: [PATCH 035/331] Upgrading to 0.130-tw-0.10 --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 36 files changed, 37 insertions(+), 37 deletions(-) diff --git a/pom.xml b/pom.xml index c627fc780d5a..45f1aab240a9 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.130-tw-0.9 + 0.130-tw-0.10 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index b1cdc92b7133..254d1a0a97df 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 2c1cf1e8232d..3551e5b9ba79 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 71ac5b6b6a71..758f46b6829f 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.130-tw-0.9 + 0.130-tw-0.10 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 7edb8a4b301f..3098d1160fb0 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 70adcfa34e45..1e6d4890fab2 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 9dcd4c01479e..e7cb584603c0 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 7cee57559957..25be93acdfe5 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 0a2a80999767..5cd6892eb6ed 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index c794568a3064..32c2f9e32945 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 31a0e5011eba..593000f9eb64 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 46480d8d82e9..938bb689ee59 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 64deba3e5bc7..06b43819b485 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 134384e326a8..a23dc2923fe8 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 8bcbb3075b9d..0fc8dfc22def 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index c7d937faa290..fe3d827a1628 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index baa24b1ef8de..cfeeb3122bda 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index ff01a6845134..afe37061a74c 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index bcf29c7aa29a..19aa2a3e7f7b 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 4b195e0d7925..4e15544c3e32 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index d8639669a4bc..eb7bb0041273 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 073f5f3cafa3..58ed96057649 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index bbfefe90b54c..659f51f9e7ee 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 8801d9be3117..8b73a4081cee 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 7ff1c69c918d..1868af2c227c 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.130-tw-0.9 + 0.130-tw-0.10 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index b95b1d27d1fb..47a876a3f1a1 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 4d22fb8ad760..bedfad6e0542 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 4f95e5638e5d..67ac2b629ac3 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index db15e14079e3..d9e657b67322 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 8c6a691ba085..1369629d997b 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 618475375aaf..be03619e8409 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index cecc9e929c4f..300a83451b02 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index e3aa177c836c..b826dfab48a9 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 07ecc82a8563..0efab43b7181 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.130-tw-0.9 + 0.130-tw-0.10 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 2ce2e06261ff..34b329a0af18 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 5464d09e2217..a34d0649d123 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.9 + 0.130-tw-0.10 presto-verifier From 1b38bdf471eb43e90127200d828f8d2ea02b3101 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Mon, 4 Jan 2016 11:46:32 -0800 Subject: [PATCH 036/331] Updating unit tests for ReadAsQueryUser flag --- .../java/com/facebook/presto/hive/TestHiveClientConfig.java | 3 +++ 1 file changed, 3 insertions(+) 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 08311e3c09e1..b7647ee99677 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 @@ -66,6 +66,7 @@ public void testDefaults() .setResourceConfigFiles((String) null) .setHiveStorageFormat(HiveStorageFormat.RCBINARY) .setRespectTableFormat(true) + .setReadAsQueryUser(false) .setImmutablePartitions(false) .setMaxPartitionsPerWriter(100) .setUseParquetColumnNames(false) @@ -132,6 +133,7 @@ public void testExplicitPropertyMappings() .put("hive.max-concurrent-file-renames", "100") .put("hive.assume-canonical-partition-keys", "true") .put("hive.parquet.use-column-names", "true") + .put("hive.read-as-query-user", "true") .put("hive.s3.aws-access-key", "abc123") .put("hive.s3.aws-secret-key", "secret") .put("hive.s3.use-instance-credentials", "false") @@ -185,6 +187,7 @@ public void testExplicitPropertyMappings() .setVerifyChecksum(false) .setResourceConfigFiles(ImmutableList.of("/foo.xml", "/bar.xml")) .setHiveStorageFormat(HiveStorageFormat.SEQUENCEFILE) + .setReadAsQueryUser(true) .setRespectTableFormat(false) .setImmutablePartitions(true) .setMaxPartitionsPerWriter(222) From 9f646d1af11a8c43e350d4d4fbc4a9ca31d83019 Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Wed, 6 Jan 2016 12:33:56 -0800 Subject: [PATCH 037/331] Avoid NPE when ZK changed state is related to connection change. --- .../presto/hive/ZookeeperMetastoreMonitor.java | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperMetastoreMonitor.java b/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperMetastoreMonitor.java index 002eefe5cc13..3789ca3c3271 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperMetastoreMonitor.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperMetastoreMonitor.java @@ -94,31 +94,25 @@ private HostAndPort deserialize(byte[] bytes) @Override public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception { - String node = ZKPaths.getNodeFromPath(event.getData().getPath()); - switch (event.getType()) { - case CHILD_ADDED: { - HostAndPort hostPort = deserialize(event.getData().getData()); - log.info("child added: " + node + ": " + hostPort); - servers.put(node, hostPort); - break; - } - + case CHILD_ADDED: case CHILD_UPDATED: { HostAndPort hostPort = deserialize(event.getData().getData()); + String node = ZKPaths.getNodeFromPath(event.getData().getPath()); log.info("child updated: " + node + ": " + hostPort); servers.put(node, hostPort); break; } case CHILD_REMOVED: { + String node = ZKPaths.getNodeFromPath(event.getData().getPath()); log.info("child removed: " + node); servers.remove(node); break; } default: - log.info("connection state changed: " + node); + log.info("connection state changed: " + event.getType()); break; } } From 48973308e108c13354c5c7b33823584fd32cad90 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Tue, 12 Jan 2016 10:38:13 -0800 Subject: [PATCH 038/331] Cache the UGI object per user in BackgroundHiveSplitLoader --- .../hive/BackgroundHiveSplitLoader.java | 42 +++++++++++++++---- 1 file changed, 33 insertions(+), 9 deletions(-) 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 82b6d7cb54da..4c3c6e2e1312 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 @@ -51,6 +51,7 @@ import java.util.Optional; import java.util.Properties; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicInteger; @@ -72,6 +73,11 @@ public class BackgroundHiveSplitLoader { public static final CompletableFuture COMPLETED_FUTURE = CompletableFuture.completedFuture(null); + // Every instance of a UserGroupInformation object for a given user has a unique hashcode, due + // to the hashCode() impl. If we don't cache the UGI per-user here, there will be a memory leak + // in the PrestoFileSystemCache. + private static final Map UGI_CACHE = new ConcurrentHashMap<>(); + private final String connectorId; private final Table table; private final Optional bucket; @@ -140,8 +146,26 @@ public BackgroundHiveSplitLoader( public void start(HiveSplitSource splitSource) { this.hiveSplitSource = splitSource; + + UserGroupInformation ugi = null; + + if (HiveSessionProperties.getReadAsQueryUser(session)) { + String user = session.getUser(); + ugi = UGI_CACHE.get(user); + + if (ugi == null) { + // TODO: Configure hadoop to allow presto daemon user to impersonate all presto users + // (HADOOPINFRA-7081) and then change to the approach below (IQ-85). + // See https://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/hadoop-common/Superusers.html + // UserGroupInformation ugi = UserGroupInformation.createProxyUser( + // session.getUser(), UserGroupInformation.getLoginUser()); + ugi = UserGroupInformation.createRemoteUser(user); + UGI_CACHE.put(user, ugi); + } + } + for (int i = 0; i < maxPartitionBatchSize; i++) { - ResumableTasks.submit(executor, new HiveSplitLoaderTask()); + ResumableTasks.submit(executor, new HiveSplitLoaderTask(ugi)); } } @@ -154,18 +178,18 @@ public void stop() private class HiveSplitLoaderTask implements ResumableTask { + private UserGroupInformation ugi; + + public HiveSplitLoaderTask(UserGroupInformation ugi) + { + this.ugi = ugi; + } + @Override public TaskStatus process() { - if (HiveSessionProperties.getReadAsQueryUser(session)) { + if (ugi != null) { try { - // TODO: Configure hadoop to allow presto daemon user to impersonate all presto users - // (HADOOPINFRA-7081) and then change to the approach below (IQ-85). - // See https://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/hadoop-common/Superusers.html - // UserGroupInformation ugi = UserGroupInformation.createProxyUser( - // session.getUser(), UserGroupInformation.getLoginUser()); - UserGroupInformation ugi = UserGroupInformation.createRemoteUser(session.getUser()); - return ugi.doAs((PrivilegedExceptionAction) this::doProcess); } catch (IOException | InterruptedException e) { From 53ae65b2def1cf7c7dc64e2d46037d1869db5453 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Tue, 12 Jan 2016 11:44:36 -0800 Subject: [PATCH 039/331] Upgrading version to 0.130-tw-0.11 --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 36 files changed, 37 insertions(+), 37 deletions(-) diff --git a/pom.xml b/pom.xml index 45f1aab240a9..18a82637339e 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.130-tw-0.10 + 0.130-tw-0.11 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 254d1a0a97df..38f39e5bed6e 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 3551e5b9ba79..c0681a377e35 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 758f46b6829f..ad5f16c3d1ce 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.130-tw-0.10 + 0.130-tw-0.11 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 3098d1160fb0..9d9927732b12 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 1e6d4890fab2..57ede93b43d9 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index e7cb584603c0..d7332844143c 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 25be93acdfe5..e9448441e73f 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 5cd6892eb6ed..ac21d14d80ec 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 32c2f9e32945..854bb94bd03a 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 593000f9eb64..0c1b2d9b6de5 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 938bb689ee59..0425c49408ed 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 06b43819b485..bfcc07f1f1e8 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index a23dc2923fe8..e82ef33a456f 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 0fc8dfc22def..f5e8c4c1953f 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index fe3d827a1628..3602a338b4f5 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index cfeeb3122bda..897fbb6cad37 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index afe37061a74c..82f8ab1f75e9 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 19aa2a3e7f7b..b88f23e6c6a6 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 4e15544c3e32..9531e434a54a 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index eb7bb0041273..dd68037b9691 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 58ed96057649..3f4f329dc9e9 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 659f51f9e7ee..20cf118a617c 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 8b73a4081cee..81edf97a3c5d 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 1868af2c227c..db8f461db9f6 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.130-tw-0.10 + 0.130-tw-0.11 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 47a876a3f1a1..879bbc03bcf0 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index bedfad6e0542..224ea2174faf 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 67ac2b629ac3..d22265c2e95c 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index d9e657b67322..562371fbc934 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 1369629d997b..399083032f49 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index be03619e8409..64f4a5aae122 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 300a83451b02..a1b8f7b4b9f7 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index b826dfab48a9..0aa645c9613e 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 0efab43b7181..7332273d684a 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.130-tw-0.10 + 0.130-tw-0.11 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 34b329a0af18..bef17e132820 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index a34d0649d123..e894a2a8420b 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.10 + 0.130-tw-0.11 presto-verifier From eff466b617cc1d1dbf1f25b805b2ecdde33c184c Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Wed, 13 Jan 2016 10:56:49 -0800 Subject: [PATCH 040/331] Change version to 0.132. --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 36 files changed, 37 insertions(+), 37 deletions(-) diff --git a/pom.xml b/pom.xml index 18a82637339e..0505afdef136 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.130-tw-0.11 + 0.132 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 38f39e5bed6e..0757573fdeac 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index c0681a377e35..7cfc90dd1d09 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index ad5f16c3d1ce..4e97c070cf5b 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.130-tw-0.11 + 0.132 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 9d9927732b12..1288733d1437 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 57ede93b43d9..3dca182b2de4 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index d7332844143c..135f8845791f 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index e9448441e73f..e6b2500dbed6 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index ac21d14d80ec..892071a00abe 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 854bb94bd03a..1f88313cc127 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 0c1b2d9b6de5..ad4b4a2a0de7 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 0425c49408ed..add0c37510b7 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index bfcc07f1f1e8..8f3ed0661a7a 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index e82ef33a456f..753f94c30bdf 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index f5e8c4c1953f..a05910678f98 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 3602a338b4f5..9306604a38b3 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 897fbb6cad37..57c23c70c40a 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 82f8ab1f75e9..311ff49ce349 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index b88f23e6c6a6..b0438d65c2ae 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 9531e434a54a..d3f04d7dd2aa 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index dd68037b9691..c0a1a1a32cae 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 3f4f329dc9e9..0b88fc338d2b 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 20cf118a617c..0d05fa046030 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 81edf97a3c5d..a4009d5b00ad 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index db8f461db9f6..4423dbd7e9e0 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.130-tw-0.11 + 0.132 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 879bbc03bcf0..3b69b26d948f 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 224ea2174faf..ea37942e5e63 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index d22265c2e95c..498a260bd5d2 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 562371fbc934..2ff6d2c1a64e 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 399083032f49..9d135250631d 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 64f4a5aae122..b277f10bf706 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index a1b8f7b4b9f7..36717f601a71 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 0aa645c9613e..27a65f8c462b 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 7332273d684a..23b0093b7898 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.130-tw-0.11 + 0.132 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index bef17e132820..bcc4768dedf1 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index e894a2a8420b..7cfaddeadd23 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.130-tw-0.11 + 0.132 presto-verifier From 14fdca19c6357d2cae3bf743f6f5cce69dee38e8 Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Wed, 13 Jan 2016 11:04:41 -0800 Subject: [PATCH 041/331] Change version to 0.132-tw-0.12. --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 37 files changed, 38 insertions(+), 38 deletions(-) diff --git a/pom.xml b/pom.xml index 09213bb52fad..0a7310f24e82 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.132 + 0.132-tw-0.12 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 0757573fdeac..1baffbd7964d 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 7cfc90dd1d09..c5d1e97aed38 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 4e97c070cf5b..720c269b8da6 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.132 + 0.132-tw-0.12 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index e51c2780dd01..8e9f44465cd2 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index bbf43c188d3a..a1ca82981c41 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 3dca182b2de4..228bef817de6 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 135f8845791f..f17279106bfa 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index e6b2500dbed6..80c85709d67f 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 892071a00abe..3201c365c7eb 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 1f88313cc127..07bb8a3fedfd 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index ad4b4a2a0de7..d2c77960ab34 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index add0c37510b7..e5c4dbf0ded8 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 8f3ed0661a7a..9b50071e2147 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 753f94c30bdf..c33ce09bbf18 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index a05910678f98..062cd3b3acc7 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 9306604a38b3..9c99be4db232 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 57c23c70c40a..7d1d123ba81d 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 311ff49ce349..5ab9e91e369c 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index e5c954b4aea4..ba634e0b5580 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index d3f04d7dd2aa..f41514dd6603 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index c0a1a1a32cae..dc92ae9833ce 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 0b88fc338d2b..09c469a95f67 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 0d05fa046030..4842899667bb 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index a4009d5b00ad..345c15453b13 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 4423dbd7e9e0..a9844a05f653 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.132 + 0.132-tw-0.12 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 3b69b26d948f..d9b862a8316a 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index ea37942e5e63..ebfd82dfe86f 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 498a260bd5d2..3098dbba128e 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 2ff6d2c1a64e..c8bad3c5fedd 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 9d135250631d..c3e87fb73c03 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index b277f10bf706..80d537ff1e7f 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 36717f601a71..7021f5b56d1b 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 27a65f8c462b..2169be00f880 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 8113b83bbc4e..0e30a1f586fb 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.132 + 0.132-tw-0.12 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index bcc4768dedf1..8a103229074e 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 7cfaddeadd23..18c5b2961b4a 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132 + 0.132-tw-0.12 presto-verifier From 310441d3775ddacce9c69c98c602e75262822718 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Thu, 14 Jan 2016 23:31:43 -0800 Subject: [PATCH 042/331] HivePageSourceProvider to support query as user from workers --- .../hive/BackgroundHiveSplitLoader.java | 20 +------- .../presto/hive/HivePageSourceProvider.java | 22 +++++++++ .../facebook/presto/hive/util/UgiUtils.java | 49 +++++++++++++++++++ 3 files changed, 73 insertions(+), 18 deletions(-) create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/util/UgiUtils.java 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 4c3c6e2e1312..676b856e6f7c 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 @@ -16,6 +16,7 @@ import com.facebook.presto.hive.util.HiveFileIterator; import com.facebook.presto.hive.util.ResumableTask; import com.facebook.presto.hive.util.ResumableTasks; +import com.facebook.presto.hive.util.UgiUtils; import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.HostAddress; import com.facebook.presto.spi.predicate.TupleDomain; @@ -51,7 +52,6 @@ import java.util.Optional; import java.util.Properties; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicInteger; @@ -73,11 +73,6 @@ public class BackgroundHiveSplitLoader { public static final CompletableFuture COMPLETED_FUTURE = CompletableFuture.completedFuture(null); - // Every instance of a UserGroupInformation object for a given user has a unique hashcode, due - // to the hashCode() impl. If we don't cache the UGI per-user here, there will be a memory leak - // in the PrestoFileSystemCache. - private static final Map UGI_CACHE = new ConcurrentHashMap<>(); - private final String connectorId; private final Table table; private final Optional bucket; @@ -150,18 +145,7 @@ public void start(HiveSplitSource splitSource) UserGroupInformation ugi = null; if (HiveSessionProperties.getReadAsQueryUser(session)) { - String user = session.getUser(); - ugi = UGI_CACHE.get(user); - - if (ugi == null) { - // TODO: Configure hadoop to allow presto daemon user to impersonate all presto users - // (HADOOPINFRA-7081) and then change to the approach below (IQ-85). - // See https://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/hadoop-common/Superusers.html - // UserGroupInformation ugi = UserGroupInformation.createProxyUser( - // session.getUser(), UserGroupInformation.getLoginUser()); - ugi = UserGroupInformation.createRemoteUser(user); - UGI_CACHE.put(user, ugi); - } + ugi = UgiUtils.getUgi(session.getUser()); } for (int i = 0; i < maxPartitionBatchSize; i++) { 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 73abe1477345..ff1c81f19367 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 @@ -13,6 +13,7 @@ */ package com.facebook.presto.hive; +import com.facebook.presto.hive.util.UgiUtils; import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ConnectorPageSource; import com.facebook.presto.spi.ConnectorPageSourceProvider; @@ -26,10 +27,13 @@ import com.google.common.collect.ImmutableSet; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.security.UserGroupInformation; import org.joda.time.DateTimeZone; import javax.inject.Inject; +import java.io.IOException; +import java.security.PrivilegedExceptionAction; import java.util.List; import java.util.Optional; import java.util.Properties; @@ -67,6 +71,24 @@ public HivePageSourceProvider( @Override public ConnectorPageSource createPageSource(ConnectorSession session, ConnectorSplit split, List columns) + { + if (HiveSessionProperties.getReadAsQueryUser(session)) { + UserGroupInformation ugi = UgiUtils.getUgi(session.getUser()); + try { + return ugi.doAs((PrivilegedExceptionAction) () -> + doCreatePageSource(session, split, columns) + ); + } + catch (IOException | InterruptedException e) { + throw new RuntimeException("Could not runAs " + session.getUser(), e); + } + } + else { + return doCreatePageSource(session, split, columns); + } + } + + private ConnectorPageSource doCreatePageSource(ConnectorSession session, ConnectorSplit split, List columns) { HiveSplit hiveSplit = checkType(split, HiveSplit.class, "split"); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/util/UgiUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/util/UgiUtils.java new file mode 100644 index 000000000000..5c8e99e75efa --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/util/UgiUtils.java @@ -0,0 +1,49 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.hive.util; + +import org.apache.hadoop.security.UserGroupInformation; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Utility class to handle creating and caching the UserGroupInformation object. + */ +public class UgiUtils +{ + private UgiUtils() {} + + // Every instance of a UserGroupInformation object for a given user has a unique hashcode, due + // to the hashCode() impl. If we don't cache the UGI per-user here, there will be a memory leak + // in the PrestoFileSystemCache. + private static final Map UGI_CACHE = new ConcurrentHashMap<>(); + + public static UserGroupInformation getUgi(String user) + { + UserGroupInformation ugi = UGI_CACHE.get(user); + + if (ugi == null) { + // TODO: Configure hadoop to allow presto daemon user to impersonate all presto users + // (HADOOPINFRA-7081) and then change to the approach below (IQ-85). + // See https://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/hadoop-common/Superusers.html + // UserGroupInformation ugi = UserGroupInformation.createProxyUser( + // session.getUser(), UserGroupInformation.getLoginUser()); + ugi = UserGroupInformation.createRemoteUser(user); + UGI_CACHE.put(user, ugi); + } + + return ugi; + } +} From cab14e9c61d7c756810d7612e559e9bc0fafbc64 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Fri, 15 Jan 2016 13:21:38 -0800 Subject: [PATCH 043/331] Upgrading presto to 0.132-tw-13 --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 37 files changed, 38 insertions(+), 38 deletions(-) diff --git a/pom.xml b/pom.xml index 0a7310f24e82..2ce743f8efef 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.132-tw-0.12 + 0.132-tw-0.13 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 1baffbd7964d..2a013d29b68b 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index c5d1e97aed38..1f9ec5acebed 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 720c269b8da6..be910a6326bd 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.132-tw-0.12 + 0.132-tw-0.13 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 8e9f44465cd2..40e60d9700dc 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index a1ca82981c41..c9cb7dcf52a9 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 228bef817de6..49eb6dbe2b24 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index f17279106bfa..49626b15d567 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 80c85709d67f..82197cc42286 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 3201c365c7eb..93686b08e969 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 07bb8a3fedfd..e0149f18d330 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index d2c77960ab34..ae357e508d3b 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index e5c4dbf0ded8..11449f92c28c 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 9b50071e2147..33937c62d142 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index c33ce09bbf18..345bfa73fded 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 062cd3b3acc7..ccc723f4dac2 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 9c99be4db232..9f34b46f98e2 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 7d1d123ba81d..28f602bc8314 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 5ab9e91e369c..9da3ebf652eb 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index ba634e0b5580..e2da193b04d9 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index f41514dd6603..1d893ba7d3e4 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index dc92ae9833ce..93f1cd722e4c 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 09c469a95f67..79af769f34dd 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 4842899667bb..58bd4e5c904a 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 345c15453b13..443b69f5a434 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index a9844a05f653..47c86b3a1ba4 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.132-tw-0.12 + 0.132-tw-0.13 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index d9b862a8316a..0d10ea4c729b 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index ebfd82dfe86f..02d01b2c3d3c 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 3098dbba128e..c945bda7597c 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index c8bad3c5fedd..33b551a1306f 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index c3e87fb73c03..a01062233738 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 80d537ff1e7f..73363ed81103 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 7021f5b56d1b..175862f3fc8c 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 2169be00f880..8a36e2d5602f 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 0e30a1f586fb..dab3686e6aa7 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.132-tw-0.12 + 0.132-tw-0.13 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 8a103229074e..8158dc7e5c8e 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 18c5b2961b4a..7ddccd9f434d 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.12 + 0.132-tw-0.13 presto-verifier From cfc21131975bdfe0756aa0bf3f4d42ba820e033f Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Sun, 31 Jan 2016 04:48:12 -0800 Subject: [PATCH 044/331] Revert PR#3, case insensitive types, since it is now supported. --- .../java/com/facebook/presto/metadata/FunctionRegistry.java | 2 +- .../main/java/com/facebook/presto/type/RowParametricType.java | 2 +- .../main/java/com/facebook/presto/spi/type/TypeSignature.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/FunctionRegistry.java b/presto-main/src/main/java/com/facebook/presto/metadata/FunctionRegistry.java index 8b30e3f8b760..4ebac76ce3af 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/FunctionRegistry.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/FunctionRegistry.java @@ -508,7 +508,7 @@ private SqlFunction getRowFieldReference(String field, TypeSignature rowTypeSign checkState(rowType.getTypeSignature().getBase().equals(StandardTypes.ROW), "rowType is not a ROW type"); SqlFunction match = null; for (SqlFunction function : RowParametricType.ROW.createFunctions(rowType)) { - if (!function.getSignature().getName().equalsIgnoreCase(field)) { + if (!function.getSignature().getName().equals(field)) { continue; } checkArgument(match == null, "Ambiguous field %s in type %s", field, rowType.getDisplayName()); diff --git a/presto-main/src/main/java/com/facebook/presto/type/RowParametricType.java b/presto-main/src/main/java/com/facebook/presto/type/RowParametricType.java index bc08fd1c3045..4705d16f87a9 100644 --- a/presto-main/src/main/java/com/facebook/presto/type/RowParametricType.java +++ b/presto-main/src/main/java/com/facebook/presto/type/RowParametricType.java @@ -68,7 +68,7 @@ public List createFunctions(Type type) RowField field = fields.get(i); int index = i; field.getName() - .ifPresent(name -> builder.add(new RowFieldReference(rowType, field.getType(), index, field.getName().get().toLowerCase()))); + .ifPresent(name -> builder.add(new RowFieldReference(rowType, field.getType(), index, field.getName().get()))); } return builder.build(); } diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/type/TypeSignature.java b/presto-spi/src/main/java/com/facebook/presto/spi/type/TypeSignature.java index 75ffe4906f81..8e5d924fe399 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/type/TypeSignature.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/type/TypeSignature.java @@ -189,7 +189,7 @@ private static Object parseLiteral(String literal) { if (literal.startsWith("'") || literal.endsWith("'")) { checkArgument(literal.startsWith("'") && literal.endsWith("'"), "Bad literal: '%s'", literal); - return literal.substring(1, literal.length() - 1).toLowerCase(); + return literal.substring(1, literal.length() - 1); } else { return Long.parseLong(literal); From b25b9f0ba1bba64009d3c393cf35ed2c60724fa1 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Tue, 2 Feb 2016 17:58:46 -0800 Subject: [PATCH 045/331] Update to use UGI.createProxyUser instead of createRemoteUser --- .../com/facebook/presto/hive/util/UgiUtils.java | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/util/UgiUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/util/UgiUtils.java index 5c8e99e75efa..08f2dde30ff2 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/util/UgiUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/util/UgiUtils.java @@ -15,6 +15,7 @@ import org.apache.hadoop.security.UserGroupInformation; +import java.io.IOException; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -35,12 +36,14 @@ public static UserGroupInformation getUgi(String user) UserGroupInformation ugi = UGI_CACHE.get(user); if (ugi == null) { - // TODO: Configure hadoop to allow presto daemon user to impersonate all presto users - // (HADOOPINFRA-7081) and then change to the approach below (IQ-85). + // Configure hadoop to allow presto daemon user to impersonate all presto users // See https://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/hadoop-common/Superusers.html - // UserGroupInformation ugi = UserGroupInformation.createProxyUser( - // session.getUser(), UserGroupInformation.getLoginUser()); - ugi = UserGroupInformation.createRemoteUser(user); + try { + ugi = UserGroupInformation.createProxyUser(user, UserGroupInformation.getLoginUser()); + } + catch (IOException e) { + throw new RuntimeException("Could not get login user from UserGroupInformation", e); + } UGI_CACHE.put(user, ugi); } From ea2c8c36829f4cf7971e7f27555c0157bc126534 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Wed, 3 Feb 2016 11:30:26 -0800 Subject: [PATCH 046/331] Upgrade to 0.132-tw-0.14 --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 37 files changed, 38 insertions(+), 38 deletions(-) diff --git a/pom.xml b/pom.xml index 2ce743f8efef..2416e615dfbe 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.132-tw-0.13 + 0.132-tw-0.14 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 2a013d29b68b..1f6bbc8b9648 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 1f9ec5acebed..114bf047c473 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index be910a6326bd..339f20cd8a98 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.132-tw-0.13 + 0.132-tw-0.14 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 40e60d9700dc..83e0d9c4fbc3 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index c9cb7dcf52a9..8dbaae2f8ae7 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 49eb6dbe2b24..7d5c36b52a5f 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 49626b15d567..68ca8e164066 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 82197cc42286..766de9cc7394 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 93686b08e969..d6b9e4b865f0 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index e0149f18d330..4383a60d3042 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index ae357e508d3b..2bd05749d80b 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 11449f92c28c..a795a1c8141d 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 33937c62d142..0339fdc41b7c 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 345bfa73fded..861da65b5d72 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index ccc723f4dac2..83d79d919e62 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 9f34b46f98e2..97675a00841a 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 28f602bc8314..af34740a897f 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 9da3ebf652eb..c5511c6b3d64 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index e2da193b04d9..9fa2fef79ff5 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 1d893ba7d3e4..48948e33ab0b 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 93f1cd722e4c..94fff311cc77 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 79af769f34dd..5749e3cfca24 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 58bd4e5c904a..a84aeb8f9a8c 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 443b69f5a434..c15dd15cd386 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 47c86b3a1ba4..3f982d5f5342 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.132-tw-0.13 + 0.132-tw-0.14 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 0d10ea4c729b..5630396e0e01 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 02d01b2c3d3c..4689f17b5d6c 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index c945bda7597c..47ee9d810e78 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 33b551a1306f..92a6f8c8ce62 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index a01062233738..3d7badbc6e9b 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 73363ed81103..c5c7ee7d4f3e 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 175862f3fc8c..4b718d149db5 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 8a36e2d5602f..c5aaefea41c0 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index dab3686e6aa7..fbe89c2ee7f0 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.132-tw-0.13 + 0.132-tw-0.14 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 8158dc7e5c8e..9f39f8e46511 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 7ddccd9f434d..d1ee25a4fac7 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.13 + 0.132-tw-0.14 presto-verifier From 6ac4bc4686e2474eaf7804554e50fdae91458843 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Wed, 3 Feb 2016 16:47:44 -0800 Subject: [PATCH 047/331] Whitelist zeppelin temporarily --- .../main/java/com/facebook/presto/hive/util/UgiUtils.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/util/UgiUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/util/UgiUtils.java index 08f2dde30ff2..d5af9fb53d4c 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/util/UgiUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/util/UgiUtils.java @@ -39,7 +39,13 @@ public static UserGroupInformation getUgi(String user) // Configure hadoop to allow presto daemon user to impersonate all presto users // See https://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/hadoop-common/Superusers.html try { - ugi = UserGroupInformation.createProxyUser(user, UserGroupInformation.getLoginUser()); + // TODO: IQ-152 roll this back once zeppelin runsAs user + if ("zeppelin".equals(user)) { + ugi = UserGroupInformation.createRemoteUser(user); + } + else { + ugi = UserGroupInformation.createProxyUser(user, UserGroupInformation.getLoginUser()); + } } catch (IOException e) { throw new RuntimeException("Could not get login user from UserGroupInformation", e); From 6d857e619eab832f58ef868bf2aa9d414f9d42cf Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Wed, 3 Feb 2016 17:13:39 -0800 Subject: [PATCH 048/331] Upgrade to 0.132-tw-0.15 --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 37 files changed, 38 insertions(+), 38 deletions(-) diff --git a/pom.xml b/pom.xml index 2416e615dfbe..f14af7bda23b 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.132-tw-0.14 + 0.132-tw-0.15 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 1f6bbc8b9648..bf932e890e47 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 114bf047c473..b0115ab5c45a 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 339f20cd8a98..787722a101b6 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.132-tw-0.14 + 0.132-tw-0.15 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 83e0d9c4fbc3..afa61e58fbe7 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 8dbaae2f8ae7..b0ef31fe590b 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 7d5c36b52a5f..c07316019a33 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 68ca8e164066..410fd2ea0307 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 766de9cc7394..6816ba6073ab 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index d6b9e4b865f0..3ab195df116b 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 4383a60d3042..363e6ec51409 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 2bd05749d80b..84d401d9faa7 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index a795a1c8141d..a7a279c4a475 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 0339fdc41b7c..5f0851c1f5d0 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 861da65b5d72..431cee305c72 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 83d79d919e62..610809a45d5e 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 97675a00841a..65ef3e4a3779 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index af34740a897f..39edd5685be9 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index c5511c6b3d64..2ca95314cfef 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 9fa2fef79ff5..0932dae34401 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 48948e33ab0b..4e331b4bb39a 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 94fff311cc77..ff77684f5e9d 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 5749e3cfca24..f6b84dc54dc0 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index a84aeb8f9a8c..9ed3bf4c0d63 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index c15dd15cd386..331e7c2d27d1 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 3f982d5f5342..95fce0f974a5 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.132-tw-0.14 + 0.132-tw-0.15 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 5630396e0e01..f3e78e9044d3 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 4689f17b5d6c..adf30dff5fb0 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 47ee9d810e78..73cfbf6524a5 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 92a6f8c8ce62..fee591fefc2b 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 3d7badbc6e9b..cac03417ee1b 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index c5c7ee7d4f3e..fcc758ea371f 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 4b718d149db5..49fdfa2d07c0 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index c5aaefea41c0..9ef399614934 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index fbe89c2ee7f0..c2614a98b3cb 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.132-tw-0.14 + 0.132-tw-0.15 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 9f39f8e46511..12409b46a8d4 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index d1ee25a4fac7..b4c38be3a538 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.14 + 0.132-tw-0.15 presto-verifier From 207a2cca84a79e348ea43a277add1d44981e7605 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Thu, 11 Feb 2016 14:07:07 -0800 Subject: [PATCH 049/331] Rolling back Zeppelin auth hack, add some logging --- .../java/com/facebook/presto/hive/util/UgiUtils.java | 8 +------- .../java/com/facebook/presto/server/HttpRemoteTask.java | 9 +++++++++ 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/util/UgiUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/util/UgiUtils.java index d5af9fb53d4c..08f2dde30ff2 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/util/UgiUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/util/UgiUtils.java @@ -39,13 +39,7 @@ public static UserGroupInformation getUgi(String user) // Configure hadoop to allow presto daemon user to impersonate all presto users // See https://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/hadoop-common/Superusers.html try { - // TODO: IQ-152 roll this back once zeppelin runsAs user - if ("zeppelin".equals(user)) { - ugi = UserGroupInformation.createRemoteUser(user); - } - else { - ugi = UserGroupInformation.createProxyUser(user, UserGroupInformation.getLoginUser()); - } + ugi = UserGroupInformation.createProxyUser(user, UserGroupInformation.getLoginUser()); } catch (IOException e) { throw new RuntimeException("Could not get login user from UserGroupInformation", e); diff --git a/presto-main/src/main/java/com/facebook/presto/server/HttpRemoteTask.java b/presto-main/src/main/java/com/facebook/presto/server/HttpRemoteTask.java index c21451070dd1..529b337add8b 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/HttpRemoteTask.java +++ b/presto-main/src/main/java/com/facebook/presto/server/HttpRemoteTask.java @@ -53,6 +53,7 @@ import io.airlift.http.client.HttpClient; import io.airlift.http.client.HttpStatus; import io.airlift.http.client.Request; +import io.airlift.http.client.StaticBodyGenerator; import io.airlift.http.client.StatusResponseHandler.StatusResponse; import io.airlift.json.JsonCodec; import io.airlift.log.Logger; @@ -454,6 +455,14 @@ private synchronized void scheduleUpdate() updateErrorTracker.startRequest(); + // TODO: (billg) remove this logging or contribute it upstream + if (log.isDebugEnabled()) { + String size = "unknown"; + if (request.getBodyGenerator() instanceof StaticBodyGenerator) { + size = Integer.toString(((StaticBodyGenerator) request.getBodyGenerator()).getBody().length); + } + log.debug(String.format("scheduleUpdate POST %s, bodySize=%s sourcesSize=%s", request.getUri(), size, sources.size())); + } ListenableFuture> future = httpClient.executeAsync(request, createFullJsonResponseHandler(taskInfoCodec)); currentRequest = future; currentRequestStartNanos = System.nanoTime(); From daa37735883f0d2b2220a77664f2919bf7dd4cd4 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Thu, 11 Feb 2016 15:40:29 -0800 Subject: [PATCH 050/331] Upgrading to 0.132-tw-0.16 --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 37 files changed, 38 insertions(+), 38 deletions(-) diff --git a/pom.xml b/pom.xml index f14af7bda23b..f22df98fd73e 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.132-tw-0.15 + 0.132-tw-0.16 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index bf932e890e47..632f2069eff1 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index b0115ab5c45a..cc0eee6d5965 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 787722a101b6..05bf2895362a 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.132-tw-0.15 + 0.132-tw-0.16 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index afa61e58fbe7..0a271e785de8 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index b0ef31fe590b..6425232da79b 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index c07316019a33..8aa9693ec1da 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 410fd2ea0307..121e534b2954 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 6816ba6073ab..ece17fc3749b 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 3ab195df116b..f25c91f065bd 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 363e6ec51409..6c0df00ff71c 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 84d401d9faa7..4068bf2c3f06 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index a7a279c4a475..455d8aa2580c 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 5f0851c1f5d0..ac7c172f3d98 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 431cee305c72..fd48780e8e96 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 610809a45d5e..f9c8aeeb7330 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 65ef3e4a3779..47bd019010f6 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 39edd5685be9..c63e02329855 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 2ca95314cfef..f67ddcb8f3c9 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 0932dae34401..eb67f46ce25a 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 4e331b4bb39a..09846fc4988d 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index ff77684f5e9d..f8fc748cb600 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index f6b84dc54dc0..47c4e2b73fe3 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 9ed3bf4c0d63..2bcb0bc67c03 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 331e7c2d27d1..4ca861f83971 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 95fce0f974a5..45afe82f6e0c 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.132-tw-0.15 + 0.132-tw-0.16 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index f3e78e9044d3..af428336232c 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index adf30dff5fb0..371a08a68b28 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 73cfbf6524a5..2d440900ff36 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index fee591fefc2b..4c77ae0aaeed 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index cac03417ee1b..09f7c9e30b0f 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index fcc758ea371f..06b939aef70d 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 49fdfa2d07c0..1d365f90c836 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 9ef399614934..50270820bc3c 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index c2614a98b3cb..d82075da681d 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.132-tw-0.15 + 0.132-tw-0.16 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 12409b46a8d4..846d8f330385 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index b4c38be3a538..ff14bb0ff9ee 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.15 + 0.132-tw-0.16 presto-verifier From f4fed25634728d7a03010e5073b2614b213a8a36 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Wed, 24 Feb 2016 09:38:56 -0800 Subject: [PATCH 051/331] Change version to 0.139 to not conflict with merge --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 37 files changed, 38 insertions(+), 38 deletions(-) diff --git a/pom.xml b/pom.xml index f22df98fd73e..afb115e56e7b 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.132-tw-0.16 + 0.139 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 632f2069eff1..721e5197c68d 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index cc0eee6d5965..bee89f2aca0c 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 05bf2895362a..e6b9dfe148ef 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.132-tw-0.16 + 0.139 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 0a271e785de8..24d70e0bd523 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 6425232da79b..3cbadb8cb50c 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 8aa9693ec1da..eee77f26b681 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 121e534b2954..053443888316 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index ece17fc3749b..7e609c58d169 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index f25c91f065bd..bbdb2780c86f 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 6c0df00ff71c..74cc530dc2d6 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 4068bf2c3f06..3909da01a3e5 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 455d8aa2580c..3617329133c4 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index ac7c172f3d98..b99adacef6d4 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index fd48780e8e96..5ebdcab6d24a 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index f9c8aeeb7330..db7651abb449 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 47bd019010f6..77e51c1592c6 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index c63e02329855..459740752199 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index f67ddcb8f3c9..cd5b16486e33 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index eb67f46ce25a..881acee972aa 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 09846fc4988d..2691474bab1b 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index f8fc748cb600..f7f94917820f 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 47c4e2b73fe3..e52248e66964 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 2bcb0bc67c03..bf2af5acd118 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 4ca861f83971..54d95b03d34b 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 45afe82f6e0c..4c01f1eb9160 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.132-tw-0.16 + 0.139 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index af428336232c..4248921cc55a 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 371a08a68b28..7b6d3f420fa9 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 2d440900ff36..7b545ce1b177 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 4c77ae0aaeed..2a8ef6926ffc 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 09f7c9e30b0f..7e3151746a48 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 06b939aef70d..6460358c5dba 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 1d365f90c836..0ab884da5c84 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 50270820bc3c..cc0f53d979cb 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index d82075da681d..6d2ca1720a56 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.132-tw-0.16 + 0.139 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 846d8f330385..db45c2072959 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index ff14bb0ff9ee..adba2eee4163 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.132-tw-0.16 + 0.139 presto-verifier From 2c9da6093b6e72f8634b7c874b517dc6e729ad31 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Wed, 24 Feb 2016 10:12:18 -0800 Subject: [PATCH 052/331] upgrade pom version to 0.139-tw-0.17 --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 37 files changed, 38 insertions(+), 38 deletions(-) diff --git a/pom.xml b/pom.xml index 67358e743bdc..632225d50fe1 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.139 + 0.139-tw-0.17 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 721e5197c68d..d84fe856c920 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index bee89f2aca0c..80781a476f81 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index e6b9dfe148ef..073888296318 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.139 + 0.139-tw-0.17 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 24d70e0bd523..8117fa6549ae 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 3cbadb8cb50c..5493f10852bd 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index eee77f26b681..4339db66dc64 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 053443888316..dd2800d32a73 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 4e7fc011a28b..907443ee18ec 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index bbdb2780c86f..7df4d2893179 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 74cc530dc2d6..85b2c4f1524d 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 3909da01a3e5..1c254fb609c8 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 3617329133c4..0e58e0b8a334 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index b99adacef6d4..f75f642ef484 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 5ebdcab6d24a..22e73c96b1d0 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index db7651abb449..cc32b3878947 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 77e51c1592c6..9f748e0114ee 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 803afb3bbbd0..8be6aa536b7c 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index cd5b16486e33..ef29485ce5c9 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 73bdb896bc4d..749ca56c4191 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 2691474bab1b..34b47c0df123 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index f7f94917820f..7754bb50b25c 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index e52248e66964..422be0c545d9 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index bf2af5acd118..48218aa78fb5 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 54d95b03d34b..5891c6b4c875 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 4c01f1eb9160..6c78d18b97ca 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.139 + 0.139-tw-0.17 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index e1959313da8a..8504a4df3bc8 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 7b6d3f420fa9..0719269862f1 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 7b545ce1b177..e15a8e9be8c5 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 2a8ef6926ffc..23bd57d1aef9 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 7e3151746a48..eb130f83e50d 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 6460358c5dba..f23575bcf202 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 0ab884da5c84..85bc734c1bf3 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index cc0f53d979cb..97a45f3fce0f 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 6d2ca1720a56..75edb19713ae 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.139 + 0.139-tw-0.17 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index db45c2072959..730294962ce5 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index ad246ed83646..507e3ebe0c51 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139 + 0.139-tw-0.17 presto-verifier From 582df38cca94dbe0f447673c34973badc96397e6 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Wed, 24 Feb 2016 14:48:28 -0800 Subject: [PATCH 053/331] Adding query logging to presto queries --- .../java/com/facebook/presto/execution/SqlQueryManager.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryManager.java b/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryManager.java index e004f2024012..52a0751cc3d1 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryManager.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryManager.java @@ -306,6 +306,10 @@ public QueryInfo createQuery(Session session, String query) stats.queryFinished(info); queryMonitor.completionEvent(info); expirationQueue.add(queryExecution); + log.info(String.format("Query complete\t%s\t%s\t%s\t%s\t%s\t%s\t%s", + info.getQueryId(), newValue, info.getErrorType(), info.getErrorCode(), + session.getUser(), info.getQueryStats().getElapsedTime(), + info.getQuery().replace(System.getProperty("line.separator"), " "))); } }); From 04e60cce32c2a06bf77818fb6b522c0ab0864eea Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Mon, 29 Feb 2016 14:38:01 -0800 Subject: [PATCH 054/331] Add a few logs in DiscoveryNodeManager to debug 'No worker nodes' error. --- .../facebook/presto/metadata/DiscoveryNodeManager.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/DiscoveryNodeManager.java b/presto-main/src/main/java/com/facebook/presto/metadata/DiscoveryNodeManager.java index f6d470e88352..7238f4f3f94b 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/DiscoveryNodeManager.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/DiscoveryNodeManager.java @@ -26,6 +26,7 @@ import io.airlift.discovery.client.ServiceSelector; import io.airlift.discovery.client.ServiceType; import io.airlift.http.client.HttpClient; +import io.airlift.log.Logger; import io.airlift.node.NodeInfo; import io.airlift.units.Duration; @@ -60,6 +61,8 @@ public final class DiscoveryNodeManager implements InternalNodeManager { + private static final Logger log = Logger.get(DiscoveryNodeManager.class); + private static final Duration MAX_AGE = new Duration(5, TimeUnit.SECONDS); private static final Splitter DATASOURCES_SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings(); @@ -124,12 +127,18 @@ public void startPollingNodeStates() Set deadNodes = difference(nodeStates.keySet(), aliveNodeIds).immutableCopy(); nodeStates.keySet().removeAll(deadNodes); + if (deadNodes.size() > 0) { + log.warn("Dead nodes: %s", deadNodes); + } + // Add new nodes for (Node node : aliveNodes) { nodeStates.putIfAbsent(node.getNodeIdentifier(), new RemoteNodeState(httpClient, uriBuilderFrom(node.getHttpUri()).appendPath("/v1/info/state").build())); } + log.debug("Number of alive nodes: %d", nodeStates.size()); + // Schedule refresh nodeStates.values().forEach(RemoteNodeState::asyncRefresh); }, 1, 5, TimeUnit.SECONDS); From 4ca01a760e2610e19562eea4c4e2e5ff749283c9 Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Mon, 29 Feb 2016 15:27:05 -0800 Subject: [PATCH 055/331] Upgrade to 0.139-tw-0.18 --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 37 files changed, 38 insertions(+), 38 deletions(-) diff --git a/pom.xml b/pom.xml index 632225d50fe1..9f3597db7a77 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.139-tw-0.17 + 0.139-tw-0.18 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index d84fe856c920..1b0a10a8cbd6 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 80781a476f81..1629008c4a62 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 073888296318..671b065d02f8 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.139-tw-0.17 + 0.139-tw-0.18 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 8117fa6549ae..fa8e9ef3fc0e 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 5493f10852bd..031b25c708c4 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 4339db66dc64..b9c5632037af 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index dd2800d32a73..a8b37a95d835 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 907443ee18ec..23b87a6a0851 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 7df4d2893179..0033399e5eb7 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 85b2c4f1524d..9a212eb99757 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 1c254fb609c8..fafbe305fc71 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 0e58e0b8a334..f2ef547e83e6 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index f75f642ef484..9b784c66bf4b 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 22e73c96b1d0..a984ac81da6e 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index cc32b3878947..02f321b1d5e3 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 9f748e0114ee..9f1258948e5d 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 8be6aa536b7c..09d2e4ec341c 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index ef29485ce5c9..cc0875536dae 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 749ca56c4191..498eaaabdfc8 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 34b47c0df123..a53264ad7f8b 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 7754bb50b25c..9b547152999b 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 422be0c545d9..7ded555176ad 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 48218aa78fb5..518f2dd91b55 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 5891c6b4c875..aed0b1c31591 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 6c78d18b97ca..6b1a9d92df35 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.139-tw-0.17 + 0.139-tw-0.18 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 8504a4df3bc8..e96cab0b52f0 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 0719269862f1..88ea917ca2d8 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index e15a8e9be8c5..e38908baa260 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 23bd57d1aef9..94886cb04ccc 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index eb130f83e50d..84c31bc33f0f 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index f23575bcf202..edfa5063e36a 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 85bc734c1bf3..4c737d4e9be3 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 97a45f3fce0f..83afc4fe5eaa 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 75edb19713ae..aac5464df7de 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.139-tw-0.17 + 0.139-tw-0.18 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 730294962ce5..539fb12adb9a 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 507e3ebe0c51..d81371230049 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.17 + 0.139-tw-0.18 presto-verifier From f1085e08d059e10ec18706e1a0927e31bce34edc Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Tue, 1 Mar 2016 12:16:25 -0800 Subject: [PATCH 056/331] Look up parquet columns by name case-insensitive --- .../facebook/presto/hive/parquet/ParquetTypeUtils.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java index bcf9027daeba..0fce1d81eced 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java @@ -15,6 +15,7 @@ import com.facebook.presto.hive.HiveColumnHandle; import parquet.schema.MessageType; +import parquet.schema.Type; public final class ParquetTypeUtils { @@ -28,6 +29,13 @@ public static parquet.schema.Type getParquetType(HiveColumnHandle column, Messag if (messageType.containsField(column.getName())) { return messageType.getType(column.getName()); } + // parquet is case-sensitive, but hive is not. all hive columns get converted to lowercase + // check for direct match above but if no match found, try case-insensitive match + for (Type type : messageType.getFields()) { + if (type.getName().equalsIgnoreCase(column.getName())) { + return type; + } + } return null; } From 5b4647a1bb4e8638d38e9e376469bec5ae26ee27 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Tue, 1 Mar 2016 16:47:02 -0800 Subject: [PATCH 057/331] Upgrading to 0.139-tw-0.19 --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 37 files changed, 38 insertions(+), 38 deletions(-) diff --git a/pom.xml b/pom.xml index 9f3597db7a77..f9431fab7e4c 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.139-tw-0.18 + 0.139-tw-0.19 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 1b0a10a8cbd6..48bdb7c717c6 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 1629008c4a62..9b83d6abcd3d 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 671b065d02f8..55598291a33c 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.139-tw-0.18 + 0.139-tw-0.19 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index fa8e9ef3fc0e..9491d2c60eaa 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 031b25c708c4..47d7c6fef678 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index b9c5632037af..c4fd099e44d0 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index a8b37a95d835..9ae21a132d6f 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 23b87a6a0851..64003da568a8 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 0033399e5eb7..508ba4cca260 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 9a212eb99757..5bb50b9fedf9 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index fafbe305fc71..85a7d855d03d 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index f2ef547e83e6..c796d9bc36d5 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 9b784c66bf4b..8784ec38bc75 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index a984ac81da6e..83cf942d5b8d 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 02f321b1d5e3..19c29f7401b8 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 9f1258948e5d..4d3016e99c94 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 09d2e4ec341c..a85718e4708a 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index cc0875536dae..1fe37919474c 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 498eaaabdfc8..b04fed59a0ff 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index a53264ad7f8b..5359608986e9 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 9b547152999b..30a708b8d147 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 7ded555176ad..74512ff0e849 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 518f2dd91b55..0d57737392af 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index aed0b1c31591..6384f163283e 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 6b1a9d92df35..693d2bb9c135 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.139-tw-0.18 + 0.139-tw-0.19 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index e96cab0b52f0..2952839013c2 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 88ea917ca2d8..53c00bb94f9a 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index e38908baa260..490cd8218967 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 94886cb04ccc..da1094e39d4b 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 84c31bc33f0f..de827fc9fd70 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index edfa5063e36a..33c89198a37c 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 4c737d4e9be3..c627552ef199 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 83afc4fe5eaa..083bb3564d3c 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index aac5464df7de..0d21f45e190e 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.139-tw-0.18 + 0.139-tw-0.19 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 539fb12adb9a..b10f3a011cab 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index d81371230049..1a30c0800d0d 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.18 + 0.139-tw-0.19 presto-verifier From f766a0c616d437eca61bb9b4f1d9e7712373341c Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Tue, 1 Mar 2016 18:07:59 -0800 Subject: [PATCH 058/331] Handle hive keywords when doing a name-based parquet field lookup --- .../presto/hive/parquet/ParquetTypeUtils.java | 34 +++++++++++++------ 1 file changed, 24 insertions(+), 10 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java index 0fce1d81eced..8891fb88afcd 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java @@ -26,17 +26,15 @@ private ParquetTypeUtils() public static parquet.schema.Type getParquetType(HiveColumnHandle column, MessageType messageType, boolean useParquetColumnNames) { if (useParquetColumnNames) { - if (messageType.containsField(column.getName())) { - return messageType.getType(column.getName()); - } - // parquet is case-sensitive, but hive is not. all hive columns get converted to lowercase - // check for direct match above but if no match found, try case-insensitive match - for (Type type : messageType.getFields()) { - if (type.getName().equalsIgnoreCase(column.getName())) { - return type; - } + String name = column.getName(); + Type type = getParquetTypeByName(name, messageType); + + // when a parquet field is a hive keyword we append an _ to it in hive. When doing + // a name-based lookup, we need to strip it off again if we didn't get a direct match. + if (type == null && name.endsWith("_")) { + type = getParquetTypeByName(name.substring(0, name.length() - 1), messageType); } - return null; + return type; } if (column.getHiveColumnIndex() < messageType.getFieldCount()) { @@ -44,4 +42,20 @@ public static parquet.schema.Type getParquetType(HiveColumnHandle column, Messag } return null; } + + private static parquet.schema.Type getParquetTypeByName(String columnName, MessageType messageType) + { + if (messageType.containsField(columnName)) { + return messageType.getType(columnName); + } + // parquet is case-sensitive, but hive is not. all hive columns get converted to lowercase + // check for direct match above but if no match found, try case-insensitive match + for (Type type : messageType.getFields()) { + if (type.getName().equalsIgnoreCase(columnName)) { + return type; + } + } + + return null; + } } From ca9961e8deeca4f1640673511870fbb08d908b72 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Tue, 1 Mar 2016 20:40:26 -0800 Subject: [PATCH 059/331] Refactor to make diff cleaner --- .../presto/hive/parquet/ParquetTypeUtils.java | 32 +++++++++++++------ 1 file changed, 23 insertions(+), 9 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java index 8891fb88afcd..bdd315f17218 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java @@ -26,15 +26,7 @@ private ParquetTypeUtils() public static parquet.schema.Type getParquetType(HiveColumnHandle column, MessageType messageType, boolean useParquetColumnNames) { if (useParquetColumnNames) { - String name = column.getName(); - Type type = getParquetTypeByName(name, messageType); - - // when a parquet field is a hive keyword we append an _ to it in hive. When doing - // a name-based lookup, we need to strip it off again if we didn't get a direct match. - if (type == null && name.endsWith("_")) { - type = getParquetTypeByName(name.substring(0, name.length() - 1), messageType); - } - return type; + return findParquetTypeByName(column, messageType); } if (column.getHiveColumnIndex() < messageType.getFieldCount()) { @@ -43,6 +35,28 @@ public static parquet.schema.Type getParquetType(HiveColumnHandle column, Messag return null; } + /** + * Find the column type by name using returning the first match with the following logic: + *
    + *
  • direct match
  • + *
  • case-insensitive match
  • + *
  • if the name ends with _, remove it and direct match
  • + *
  • if the name ends with _, remove it and case-insensitive match
  • + *
+ */ + private static parquet.schema.Type findParquetTypeByName(HiveColumnHandle column, MessageType messageType) + { + String name = column.getName(); + Type type = getParquetTypeByName(name, messageType); + + // when a parquet field is a hive keyword we append an _ to it in hive. When doing + // a name-based lookup, we need to strip it off again if we didn't get a direct match. + if (type == null && name.endsWith("_")) { + type = getParquetTypeByName(name.substring(0, name.length() - 1), messageType); + } + return type; + } + private static parquet.schema.Type getParquetTypeByName(String columnName, MessageType messageType) { if (messageType.containsField(columnName)) { From d7d66a28954049e9d41baf0d7eefd0bfd245637e Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Fri, 11 Mar 2016 11:25:21 -0800 Subject: [PATCH 060/331] Upgrade to 0.141. --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 37 files changed, 38 insertions(+), 38 deletions(-) diff --git a/pom.xml b/pom.xml index f9431fab7e4c..1cc3360c33ef 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.139-tw-0.19 + 0.141 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 48bdb7c717c6..aefbe525d11c 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 9b83d6abcd3d..883a041146d2 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 55598291a33c..546f6323162c 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.139-tw-0.19 + 0.141 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 9491d2c60eaa..7637c11bcb13 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 47d7c6fef678..f24bb402f6f5 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index c4fd099e44d0..db5650329053 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 9ae21a132d6f..755f4b0c0bd0 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 64003da568a8..1bff918f4e79 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 508ba4cca260..ad3c2de908f1 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 5bb50b9fedf9..5bd58c4889a2 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 85a7d855d03d..360175ba7f1f 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index c796d9bc36d5..fc5b3c6bb0cc 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 8784ec38bc75..2e2d0c2e6938 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 83cf942d5b8d..f9cedd62b887 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 19c29f7401b8..77c66becb7c1 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 4d3016e99c94..80c89914cb60 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index a85718e4708a..8271359203ce 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 1fe37919474c..057819d9734b 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index b04fed59a0ff..ecbbd667febc 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 5359608986e9..1aa7767621ae 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 30a708b8d147..f02eced794de 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 74512ff0e849..906c40b97ba9 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 0d57737392af..b30f2e739442 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 6384f163283e..8f130d06d809 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 693d2bb9c135..ec7b54712bed 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.139-tw-0.19 + 0.141 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 2952839013c2..17601b1b1823 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 53c00bb94f9a..f6f597487812 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 490cd8218967..31eb7a942001 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index da1094e39d4b..eb44ebac42f7 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index de827fc9fd70..67b64fdc2156 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 33c89198a37c..5b61cec72b21 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index c627552ef199..e9ab9fba2bba 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 083bb3564d3c..05bdf43a0a08 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 0d21f45e190e..696256a3122b 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.139-tw-0.19 + 0.141 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index b10f3a011cab..72aaa2acb3dd 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 1a30c0800d0d..5f6c00cf18f5 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.139-tw-0.19 + 0.141 presto-verifier From 94bcf3e46c886684e45a5a796d0ae9167e5ad6f8 Mon Sep 17 00:00:00 2001 From: Sailesh Mittal Date: Fri, 11 Mar 2016 11:29:59 -0800 Subject: [PATCH 061/331] Upgrade to 0.141-tw-0.20. --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 37 files changed, 38 insertions(+), 38 deletions(-) diff --git a/pom.xml b/pom.xml index 411996e53ee0..130f0ff9eb9d 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.141 + 0.141-tw-0.20 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index def710c5c2c6..00cc629d364c 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 883a041146d2..3652e17bd06f 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 546f6323162c..8f86dabd3e25 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.141 + 0.141-tw-0.20 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 7637c11bcb13..1c0d41086973 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index f24bb402f6f5..1bdeaecdcea2 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index db5650329053..4f0675eda340 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 755f4b0c0bd0..bdcff6cdb591 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 1bff918f4e79..f54c94db8398 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index ad3c2de908f1..4b4ad310d3e1 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 5bd58c4889a2..2c4cc361f6fb 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 360175ba7f1f..7bb7379f70b6 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index fc5b3c6bb0cc..09e13a19e68d 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 2e2d0c2e6938..6e304d8b10bd 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index f9cedd62b887..cfa615c8d2dc 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 77c66becb7c1..cebd8614c068 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 80c89914cb60..24ca263be84e 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 8271359203ce..4ae769912294 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 057819d9734b..31620ae9fe5a 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index ecbbd667febc..b1eed0066f04 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 1aa7767621ae..b5e67298855a 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index f02eced794de..4722581cd3a5 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 906c40b97ba9..d710b58886f5 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index b30f2e739442..830af643de31 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 8f130d06d809..c8e057ed14aa 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index ec7b54712bed..bfe2383a567f 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.141 + 0.141-tw-0.20 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 17601b1b1823..3b527cbaa5dc 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index f6f597487812..5c724e169c0c 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 31eb7a942001..4c226d7edc39 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index eb44ebac42f7..14038c648f0f 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 67b64fdc2156..2c4b2c1f49eb 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 5b61cec72b21..a977f2d08420 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index e9ab9fba2bba..301e4c4efe55 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 05bdf43a0a08..dc8b11426992 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 696256a3122b..25c2414ce83f 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.141 + 0.141-tw-0.20 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 72aaa2acb3dd..3979e5a18555 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 5f6c00cf18f5..9dbeee8b8252 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141 + 0.141-tw-0.20 presto-verifier From 3de1ca483acb8944e714631df79fe62a3023f450 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Thu, 24 Mar 2016 09:35:41 -0700 Subject: [PATCH 062/331] Use modules and query events for logging --- .../presto/execution/SqlQueryManager.java | 4 - .../facebook/presto/server/PrestoServer.java | 2 +- .../facebook/presto/twitter/ModuleLoader.java | 37 ++++++++ .../presto/twitter/logging/QueryLogger.java | 91 +++++++++++++++++++ 4 files changed, 129 insertions(+), 5 deletions(-) create mode 100644 presto-main/src/main/java/com/facebook/presto/twitter/ModuleLoader.java create mode 100644 presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java diff --git a/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryManager.java b/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryManager.java index 51e261aeeb10..6722c701fd5e 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryManager.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryManager.java @@ -309,10 +309,6 @@ public QueryInfo createQuery(Session session, String query) stats.queryFinished(info); queryMonitor.completionEvent(info); expirationQueue.add(queryExecution); - log.info(String.format("Query complete\t%s\t%s\t%s\t%s\t%s\t%s\t%s", - info.getQueryId(), newValue, info.getErrorType(), info.getErrorCode(), - session.getUser(), info.getQueryStats().getElapsedTime(), - info.getQuery().replace(System.getProperty("line.separator"), " "))); } }); diff --git a/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java b/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java index de138dc25cba..685a0454c73a 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java +++ b/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java @@ -154,7 +154,7 @@ public void run() protected Iterable getAdditionalModules() { - return ImmutableList.of(); + return com.facebook.presto.twitter.ModuleLoader.getAdditionalModules(); } private static void updateDatasources(Announcer announcer, Metadata metadata, ServerConfig serverConfig, NodeSchedulerConfig schedulerConfig) diff --git a/presto-main/src/main/java/com/facebook/presto/twitter/ModuleLoader.java b/presto-main/src/main/java/com/facebook/presto/twitter/ModuleLoader.java new file mode 100644 index 000000000000..aee0a6ca2825 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/twitter/ModuleLoader.java @@ -0,0 +1,37 @@ +/* + * 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.twitter; + +import com.facebook.presto.twitter.logging.QueryLogger; +import com.google.common.collect.ImmutableList; +import com.google.inject.Module; +import com.google.inject.Scopes; +import com.google.inject.multibindings.Multibinder; +import io.airlift.event.client.EventClient; + +public class ModuleLoader +{ + private ModuleLoader() + { + } + + public static Iterable getAdditionalModules() + { + return ImmutableList.of(binder -> + Multibinder.newSetBinder(binder, EventClient.class) + .addBinding() + .to(QueryLogger.class) + .in(Scopes.SINGLETON)); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java new file mode 100644 index 000000000000..863847ab73c0 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java @@ -0,0 +1,91 @@ +/* + * 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.twitter.logging; + +import com.facebook.presto.event.query.QueryCompletionEvent; +import com.facebook.presto.spi.StandardErrorCode; +import io.airlift.event.client.AbstractEventClient; +import io.airlift.event.client.EventType; +import io.airlift.log.Logger; +import io.airlift.units.Duration; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +public class QueryLogger extends AbstractEventClient +{ + private static final int MAX_QUERY_LENGTH = 1000; + private static final String DASH = "-"; + private static final String COLON = ":"; + private static final String SPACE = " "; + private static final String ELIPSIS = "..."; + + private static final Logger log = Logger.get(QueryLogger.class); + + @Override + protected void postEvent(T event) + throws IOException + { + EventType eventTypeAnnotation = event.getClass().getAnnotation(EventType.class); + if (eventTypeAnnotation == null) { + return; + } + + // other event types exist, like QueryCreatedEvent and SplitCompletionEvent + if (eventTypeAnnotation.value().equals("QueryCompletion")) { + logQueryComplete((QueryCompletionEvent) event); + } + } + + private static void logQueryComplete(QueryCompletionEvent event) + { + String errorType = DASH; + String errorCode = DASH; + if (event.getErrorCode() != null) { + errorType = StandardErrorCode.toErrorType(event.getErrorCode()).toString(); + if (event.getErrorCodeName() != null) { + errorCode = event.getErrorCodeName() + COLON + event.getErrorCode(); + } + } + + Duration duration = (new Duration( + event.getEndTime().getMillis() - + event.getCreateTime().getMillis(), TimeUnit.MILLISECONDS)) + .convertToMostSuccinctTimeUnit(); + + log.info(String.format("Query complete\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s", + event.getQueryId(), toLogValue(event.getRemoteClientAddress()), + event.getQueryState(), errorType, errorCode, + event.getUser(), duration, + cleanseAndTrimQuery(event.getQuery()))); + } + + private static String toLogValue(Object object) + { + if (object == null) { + return DASH; + } + else { + return object.toString(); + } + } + + private static String cleanseAndTrimQuery(String query) + { + if (query.length() > MAX_QUERY_LENGTH) { + query = query.substring(0, MAX_QUERY_LENGTH) + ELIPSIS; + } + return query.replace(System.getProperty("line.separator"), SPACE); + } +} From f055db69eaf6901002e0d017e85223cc5cdede80 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Thu, 24 Mar 2016 10:25:55 -0700 Subject: [PATCH 063/331] Add javadocs --- .../facebook/presto/server/PrestoServer.java | 3 +- .../facebook/presto/twitter/ModuleLoader.java | 37 ------------------- .../presto/twitter/logging/QueryLogger.java | 3 ++ 3 files changed, 5 insertions(+), 38 deletions(-) delete mode 100644 presto-main/src/main/java/com/facebook/presto/twitter/ModuleLoader.java diff --git a/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java b/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java index 685a0454c73a..78b01e8cd4fb 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java +++ b/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java @@ -24,6 +24,7 @@ import com.facebook.presto.security.AccessControlModule; import com.facebook.presto.server.security.ServerSecurityModule; import com.facebook.presto.sql.parser.SqlParserOptions; +import com.facebook.presto.twitter.TwitterModuleLoader; import com.google.common.base.Joiner; import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; @@ -154,7 +155,7 @@ public void run() protected Iterable getAdditionalModules() { - return com.facebook.presto.twitter.ModuleLoader.getAdditionalModules(); + return TwitterModuleLoader.getAdditionalModules(); } private static void updateDatasources(Announcer announcer, Metadata metadata, ServerConfig serverConfig, NodeSchedulerConfig schedulerConfig) diff --git a/presto-main/src/main/java/com/facebook/presto/twitter/ModuleLoader.java b/presto-main/src/main/java/com/facebook/presto/twitter/ModuleLoader.java deleted file mode 100644 index aee0a6ca2825..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/twitter/ModuleLoader.java +++ /dev/null @@ -1,37 +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.twitter; - -import com.facebook.presto.twitter.logging.QueryLogger; -import com.google.common.collect.ImmutableList; -import com.google.inject.Module; -import com.google.inject.Scopes; -import com.google.inject.multibindings.Multibinder; -import io.airlift.event.client.EventClient; - -public class ModuleLoader -{ - private ModuleLoader() - { - } - - public static Iterable getAdditionalModules() - { - return ImmutableList.of(binder -> - Multibinder.newSetBinder(binder, EventClient.class) - .addBinding() - .to(QueryLogger.class) - .in(Scopes.SINGLETON)); - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java index 863847ab73c0..b70eda37da7c 100644 --- a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java +++ b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java @@ -23,6 +23,9 @@ import java.io.IOException; import java.util.concurrent.TimeUnit; +/** + * Class that listens for query completion events and logs them to a file + */ public class QueryLogger extends AbstractEventClient { private static final int MAX_QUERY_LENGTH = 1000; From 3c1c7b661cf0b79abeb33448eafd9b0fe2c421b0 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Thu, 24 Mar 2016 10:45:11 -0700 Subject: [PATCH 064/331] Add javadocs --- .../presto/twitter/TwitterModuleLoader.java | 46 +++++++++++++++++++ 1 file changed, 46 insertions(+) create mode 100644 presto-main/src/main/java/com/facebook/presto/twitter/TwitterModuleLoader.java diff --git a/presto-main/src/main/java/com/facebook/presto/twitter/TwitterModuleLoader.java b/presto-main/src/main/java/com/facebook/presto/twitter/TwitterModuleLoader.java new file mode 100644 index 000000000000..44bfc995b462 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/twitter/TwitterModuleLoader.java @@ -0,0 +1,46 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.twitter; + +import com.facebook.presto.twitter.logging.QueryLogger; +import com.google.common.collect.ImmutableList; +import com.google.inject.Module; +import com.google.inject.Scopes; +import com.google.inject.multibindings.Multibinder; +import io.airlift.event.client.EventClient; + +/** + * Loader that initializes custom Twitter code to inject into Presto. Whenever + * possible we should use this pattern to inject custom functionality, since it + * makes it easier to differentiate our patches from the core OS code. + * + * If the functionality we wish to add/override isn't currently possible to via + * overriding a guice module, we should contribute the necessary modules/interfaces + * into the OS Presto code base to make it possible. + */ +public class TwitterModuleLoader +{ + private TwitterModuleLoader() + { + } + + public static Iterable getAdditionalModules() + { + return ImmutableList.of(binder -> + Multibinder.newSetBinder(binder, EventClient.class) + .addBinding() + .to(QueryLogger.class) + .in(Scopes.SINGLETON)); + } +} From d8afd6223cc9d99d3cfd430b0752b53c0cb66c75 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Thu, 24 Mar 2016 10:47:05 -0700 Subject: [PATCH 065/331] fix imports --- .../src/main/java/com/facebook/presto/server/PrestoServer.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java b/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java index 78b01e8cd4fb..4bac276668ef 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java +++ b/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java @@ -24,7 +24,6 @@ import com.facebook.presto.security.AccessControlModule; import com.facebook.presto.server.security.ServerSecurityModule; import com.facebook.presto.sql.parser.SqlParserOptions; -import com.facebook.presto.twitter.TwitterModuleLoader; import com.google.common.base.Joiner; import com.google.common.base.Splitter; import com.google.common.collect.ImmutableList; @@ -155,7 +154,7 @@ public void run() protected Iterable getAdditionalModules() { - return TwitterModuleLoader.getAdditionalModules(); + return com.facebook.presto.twitter.TwitterModuleLoader.getAdditionalModules(); } private static void updateDatasources(Announcer announcer, Metadata metadata, ServerConfig serverConfig, NodeSchedulerConfig schedulerConfig) From 251db4d7c15df61b8e86eb1bd3e21ad904ecac04 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Thu, 24 Mar 2016 11:01:00 -0700 Subject: [PATCH 066/331] add splits, rows and bytes --- .../com/facebook/presto/twitter/logging/QueryLogger.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java index b70eda37da7c..1ad52efb0d67 100644 --- a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java +++ b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java @@ -67,10 +67,10 @@ private static void logQueryComplete(QueryCompletionEvent event) event.getCreateTime().getMillis(), TimeUnit.MILLISECONDS)) .convertToMostSuccinctTimeUnit(); - log.info(String.format("Query complete\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s", + log.info(String.format("Query complete\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s", event.getQueryId(), toLogValue(event.getRemoteClientAddress()), - event.getQueryState(), errorType, errorCode, - event.getUser(), duration, + event.getQueryState(), errorType, errorCode, event.getUser(), duration, + event.getSplits(), event.getTotalRows(), event.getTotalBytes(), cleanseAndTrimQuery(event.getQuery()))); } From 95035ecf3eb7a1a9569fa9a13a15b00463992349 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Thu, 24 Mar 2016 11:56:17 -0700 Subject: [PATCH 067/331] Change to use QueryComplete --- .../com/facebook/presto/twitter/logging/QueryLogger.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java index 1ad52efb0d67..0a6f5c3e105c 100644 --- a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java +++ b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java @@ -33,6 +33,7 @@ public class QueryLogger extends AbstractEventClient private static final String COLON = ":"; private static final String SPACE = " "; private static final String ELIPSIS = "..."; + private static final String QUERY_COMPLETION = "QueryCompletion"; private static final Logger log = Logger.get(QueryLogger.class); @@ -46,7 +47,7 @@ protected void postEvent(T event) } // other event types exist, like QueryCreatedEvent and SplitCompletionEvent - if (eventTypeAnnotation.value().equals("QueryCompletion")) { + if (eventTypeAnnotation.value().equals(QUERY_COMPLETION)) { logQueryComplete((QueryCompletionEvent) event); } } @@ -67,8 +68,8 @@ private static void logQueryComplete(QueryCompletionEvent event) event.getCreateTime().getMillis(), TimeUnit.MILLISECONDS)) .convertToMostSuccinctTimeUnit(); - log.info(String.format("Query complete\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s", - event.getQueryId(), toLogValue(event.getRemoteClientAddress()), + log.info(String.format("%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s", + QUERY_COMPLETION, event.getQueryId(), toLogValue(event.getRemoteClientAddress()), event.getQueryState(), errorType, errorCode, event.getUser(), duration, event.getSplits(), event.getTotalRows(), event.getTotalBytes(), cleanseAndTrimQuery(event.getQuery()))); From dbb4c42e330e42815a097da727f26cad68d55141 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Tue, 29 Mar 2016 12:29:07 -0700 Subject: [PATCH 068/331] Add event process and handlers and scribe query completion events --- pom.xml | 55 ++++++ presto-main/pom.xml | 18 ++ .../facebook/presto/event/EventProcessor.java | 90 ++++++++++ .../event/query/QueryCompletionEvent.java | 2 +- .../presto/event/query/QueryCreatedEvent.java | 2 +- .../presto/event/query/QueryEvent.java | 22 +++ .../presto/event/query/QueryEventHandler.java | 19 ++ .../event/query/SplitCompletionEvent.java | 2 +- .../presto/twitter/TwitterModuleLoader.java | 26 ++- .../presto/twitter/logging/QueryLogger.java | 27 +-- .../presto/twitter/logging/QueryScriber.java | 169 ++++++++++++++++++ 11 files changed, 403 insertions(+), 29 deletions(-) create mode 100644 presto-main/src/main/java/com/facebook/presto/event/EventProcessor.java create mode 100644 presto-main/src/main/java/com/facebook/presto/event/query/QueryEvent.java create mode 100644 presto-main/src/main/java/com/facebook/presto/event/query/QueryEventHandler.java create mode 100644 presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java diff --git a/pom.xml b/pom.xml index 130f0ff9eb9d..ae36b8cf3d9c 100644 --- a/pom.xml +++ b/pom.xml @@ -730,6 +730,61 @@ hive-apache-jdbc 0.13.1-1 + + + + com.twitter + presto-thrift-java + 0.0.1 + + + com.twitter + util-core_2.11 + + + com.twitter + util-core-java + + + com.twitter + util-function_2.10 + + + com.twitter + util-function-java + + + commons-logging + commons-logging + + + org.scala-lang.modules + scala-parser-combinators_2.11 + + + + + com.twitter + util-logging_2.11 + 6.33.0 + + + commons-logging + commons-logging + + + + + org.scala-lang + scala-library + 2.11.7 + + + commons-logging + commons-logging + + + diff --git a/presto-main/pom.xml b/presto-main/pom.xml index b1eed0066f04..dfe5c5d08d7a 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -296,6 +296,24 @@ tpch test + + + + com.twitter + presto-thrift-java + + + com.twitter + util-logging_2.11 + + + org.apache.thrift + libthrift + + + org.scala-lang + scala-library + diff --git a/presto-main/src/main/java/com/facebook/presto/event/EventProcessor.java b/presto-main/src/main/java/com/facebook/presto/event/EventProcessor.java new file mode 100644 index 000000000000..edaabc5437de --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/event/EventProcessor.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.event; + +import com.facebook.presto.event.query.QueryCompletionEvent; +import com.facebook.presto.event.query.QueryCreatedEvent; +import com.facebook.presto.event.query.QueryEvent; +import com.facebook.presto.event.query.QueryEventHandler; +import com.facebook.presto.event.query.SplitCompletionEvent; +import com.facebook.presto.twitter.logging.QueryLogger; +import com.google.inject.Inject; +import io.airlift.event.client.AbstractEventClient; +import io.airlift.event.client.EventType; +import io.airlift.log.Logger; + +import java.io.IOException; +import java.util.Set; + +public class EventProcessor extends AbstractEventClient +{ + private static final String QUERY_CREATED = "QueryCreated"; + private static final String QUERY_COMPLETION = "QueryCompletion"; + private static final String SPLIT_COMPLETION = "SplitCompletion"; + private static final Logger log = Logger.get(QueryLogger.class); + + private Set> queryCreatedEventHandlers; + private Set> queryCompletionEventHandlers; + private Set> splitCompletionEventHandlers; + + @Inject + public EventProcessor( + Set> queryCreatedEventHandlers, + Set> queryCompletionEventHandlers, + Set> splitCompletionEventHandlers) + { + this.queryCreatedEventHandlers = queryCreatedEventHandlers; + this.queryCompletionEventHandlers = queryCompletionEventHandlers; + this.splitCompletionEventHandlers = splitCompletionEventHandlers; + } + + @Override + protected void postEvent(T event) + throws IOException + { + EventType eventTypeAnnotation = event.getClass().getAnnotation(EventType.class); + if (eventTypeAnnotation == null) { + return; + } + + String type = eventTypeAnnotation.value(); + + switch (type) { + case QUERY_CREATED: + handle(queryCreatedEventHandlers, type, (QueryCreatedEvent) event); + break; + case QUERY_COMPLETION: + handle(queryCompletionEventHandlers, type, (QueryCompletionEvent) event); + break; + case SPLIT_COMPLETION: + handle(splitCompletionEventHandlers, type, (SplitCompletionEvent) event); + break; + default: + log.warn("Unrecognized event found: " + type); + } + } + + private void handle(Set> handlers, String type, E event) + { + for (QueryEventHandler handler : handlers) { + try { + handler.handle(event); + } + catch (Throwable e) { + log.error(e, String.format( + "Exception processing %s event for query %s", type, event.getQueryId())); + } + } + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/event/query/QueryCompletionEvent.java b/presto-main/src/main/java/com/facebook/presto/event/query/QueryCompletionEvent.java index c005649e77e2..3c95ffe66ef0 100644 --- a/presto-main/src/main/java/com/facebook/presto/event/query/QueryCompletionEvent.java +++ b/presto-main/src/main/java/com/facebook/presto/event/query/QueryCompletionEvent.java @@ -31,7 +31,7 @@ @Immutable @EventType("QueryCompletion") -public class QueryCompletionEvent +public class QueryCompletionEvent implements QueryEvent { private final QueryId queryId; private final String transactionId; diff --git a/presto-main/src/main/java/com/facebook/presto/event/query/QueryCreatedEvent.java b/presto-main/src/main/java/com/facebook/presto/event/query/QueryCreatedEvent.java index 64871433ecee..6be296bba8e7 100644 --- a/presto-main/src/main/java/com/facebook/presto/event/query/QueryCreatedEvent.java +++ b/presto-main/src/main/java/com/facebook/presto/event/query/QueryCreatedEvent.java @@ -24,7 +24,7 @@ @Immutable @EventType("QueryCreated") -public class QueryCreatedEvent +public class QueryCreatedEvent implements QueryEvent { private final QueryId queryId; private final String transactionId; diff --git a/presto-main/src/main/java/com/facebook/presto/event/query/QueryEvent.java b/presto-main/src/main/java/com/facebook/presto/event/query/QueryEvent.java new file mode 100644 index 000000000000..33ca974b826c --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/event/query/QueryEvent.java @@ -0,0 +1,22 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.event.query; + +/** + * + */ +public interface QueryEvent +{ + String getQueryId(); +} diff --git a/presto-main/src/main/java/com/facebook/presto/event/query/QueryEventHandler.java b/presto-main/src/main/java/com/facebook/presto/event/query/QueryEventHandler.java new file mode 100644 index 000000000000..2aa0bd381a5f --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/event/query/QueryEventHandler.java @@ -0,0 +1,19 @@ +/* + * 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.event.query; + +public interface QueryEventHandler +{ + void handle(T event); +} diff --git a/presto-main/src/main/java/com/facebook/presto/event/query/SplitCompletionEvent.java b/presto-main/src/main/java/com/facebook/presto/event/query/SplitCompletionEvent.java index f9374c21b09d..4a9e206b9962 100644 --- a/presto-main/src/main/java/com/facebook/presto/event/query/SplitCompletionEvent.java +++ b/presto-main/src/main/java/com/facebook/presto/event/query/SplitCompletionEvent.java @@ -29,7 +29,7 @@ @Immutable @EventType("SplitCompletion") -public class SplitCompletionEvent +public class SplitCompletionEvent implements QueryEvent { private final QueryId queryId; private final StageId stageId; diff --git a/presto-main/src/main/java/com/facebook/presto/twitter/TwitterModuleLoader.java b/presto-main/src/main/java/com/facebook/presto/twitter/TwitterModuleLoader.java index 44bfc995b462..eb23a7650a09 100644 --- a/presto-main/src/main/java/com/facebook/presto/twitter/TwitterModuleLoader.java +++ b/presto-main/src/main/java/com/facebook/presto/twitter/TwitterModuleLoader.java @@ -13,10 +13,17 @@ */ package com.facebook.presto.twitter; +import com.facebook.presto.event.EventProcessor; +import com.facebook.presto.event.query.QueryCompletionEvent; +import com.facebook.presto.event.query.QueryCreatedEvent; +import com.facebook.presto.event.query.QueryEventHandler; +import com.facebook.presto.event.query.SplitCompletionEvent; import com.facebook.presto.twitter.logging.QueryLogger; +import com.facebook.presto.twitter.logging.QueryScriber; import com.google.common.collect.ImmutableList; import com.google.inject.Module; import com.google.inject.Scopes; +import com.google.inject.TypeLiteral; import com.google.inject.multibindings.Multibinder; import io.airlift.event.client.EventClient; @@ -37,10 +44,21 @@ private TwitterModuleLoader() public static Iterable getAdditionalModules() { - return ImmutableList.of(binder -> - Multibinder.newSetBinder(binder, EventClient.class) + return ImmutableList.of( + binder -> Multibinder.newSetBinder(binder, EventClient.class) .addBinding() - .to(QueryLogger.class) - .in(Scopes.SINGLETON)); + .to(EventProcessor.class) + .in(Scopes.SINGLETON), + binder -> Multibinder.newSetBinder(binder, new TypeLiteral>() {}), + binder -> Multibinder.newSetBinder(binder, new TypeLiteral>() {}), + binder -> Multibinder.newSetBinder(binder, new TypeLiteral>(){}) + .addBinding() + .to(new TypeLiteral(){}) + .in(Scopes.SINGLETON), + binder -> Multibinder.newSetBinder(binder, new TypeLiteral>(){}) + .addBinding() + .to(QueryScriber.class) + .in(Scopes.SINGLETON) + ); } } diff --git a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java index 0a6f5c3e105c..02edd8da2440 100644 --- a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java +++ b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java @@ -14,19 +14,17 @@ package com.facebook.presto.twitter.logging; import com.facebook.presto.event.query.QueryCompletionEvent; +import com.facebook.presto.event.query.QueryEventHandler; import com.facebook.presto.spi.StandardErrorCode; -import io.airlift.event.client.AbstractEventClient; -import io.airlift.event.client.EventType; import io.airlift.log.Logger; import io.airlift.units.Duration; -import java.io.IOException; import java.util.concurrent.TimeUnit; /** - * Class that listens for query completion events and logs them to a file + * Class that logs query events to a file */ -public class QueryLogger extends AbstractEventClient +public class QueryLogger implements QueryEventHandler { private static final int MAX_QUERY_LENGTH = 1000; private static final String DASH = "-"; @@ -38,21 +36,7 @@ public class QueryLogger extends AbstractEventClient private static final Logger log = Logger.get(QueryLogger.class); @Override - protected void postEvent(T event) - throws IOException - { - EventType eventTypeAnnotation = event.getClass().getAnnotation(EventType.class); - if (eventTypeAnnotation == null) { - return; - } - - // other event types exist, like QueryCreatedEvent and SplitCompletionEvent - if (eventTypeAnnotation.value().equals(QUERY_COMPLETION)) { - logQueryComplete((QueryCompletionEvent) event); - } - } - - private static void logQueryComplete(QueryCompletionEvent event) + public void handle(QueryCompletionEvent event) { String errorType = DASH; String errorCode = DASH; @@ -64,8 +48,7 @@ private static void logQueryComplete(QueryCompletionEvent event) } Duration duration = (new Duration( - event.getEndTime().getMillis() - - event.getCreateTime().getMillis(), TimeUnit.MILLISECONDS)) + event.getQueryWallTimeMs(), TimeUnit.MILLISECONDS)) .convertToMostSuccinctTimeUnit(); log.info(String.format("%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s", diff --git a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java new file mode 100644 index 000000000000..0110afdf8a61 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java @@ -0,0 +1,169 @@ +/* + * 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.twitter.logging; + +import com.facebook.presto.event.query.QueryCompletionEvent; +import com.facebook.presto.event.query.QueryEventHandler; +import com.google.common.base.Optional; +import com.twitter.logging.BareFormatter$; +import com.twitter.logging.Level; +import com.twitter.logging.QueueingHandler; +import com.twitter.logging.ScribeHandler; +import com.twitter.presto.thriftjava.QueryState; +import io.airlift.log.Logger; +import org.apache.thrift.TBase; +import org.apache.thrift.TException; +import org.apache.thrift.TSerializer; + +import java.util.Base64; +import java.util.logging.LogRecord; + +/** + * Class that scribes query completion events + */ +public class QueryScriber implements QueryEventHandler +{ + private static final String SCRIBE_CATEGORY = "test_presto_query_complete"; + private static final int MAX_QUEUE_SIZE = 1000; + + private static final Logger log = Logger.get(QueryScriber.class); + + private QueueingHandler queueingHandler; + + // TSerializer is not thread safe + private final ThreadLocal serializer = new ThreadLocal() + { + @Override protected TSerializer initialValue() + { + return new TSerializer(); + } + }; + + public QueryScriber() + { + ScribeHandler scribeHandler = new ScribeHandler( + ScribeHandler.DefaultHostname(), + ScribeHandler.DefaultPort(), + SCRIBE_CATEGORY, + ScribeHandler.DefaultBufferTime(), + ScribeHandler.DefaultConnectBackoff(), + ScribeHandler.DefaultMaxMessagesPerTransaction(), + ScribeHandler.DefaultMaxMessagesToBuffer(), + BareFormatter$.MODULE$, + scala.Option.apply((Level) null)); + queueingHandler = new QueueingHandler(scribeHandler, MAX_QUEUE_SIZE); + } + + @Override + public void handle(QueryCompletionEvent event) + { + com.twitter.presto.thriftjava.QueryCompletionEvent thriftEvent = toThriftQueryCompletionEvent(event); + Optional message = serializeThriftToString(thriftEvent); + + if (message.isPresent()) { + LogRecord logRecord = new LogRecord(Level.ALL, message.get()); + queueingHandler.publish(logRecord); + } + else { + log.warn("Unable to serialize QueryCompletionEvent: " + event); + } + } + + /** + * Serialize a thrift object to bytes, compress, then encode as a base64 string. + */ + private Optional serializeThriftToString(TBase thriftMessage) + { + try { + return Optional.of( + Base64.getEncoder().encodeToString(serializer.get().serialize(thriftMessage))); + } + catch (TException e) { + log.warn(e, "Could not serialize thrift object" + thriftMessage); + return Optional.absent(); + } + } + + private static com.twitter.presto.thriftjava.QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletionEvent event) + { + com.twitter.presto.thriftjava.QueryCompletionEvent thriftEvent = + new com.twitter.presto.thriftjava.QueryCompletionEvent(); + + thriftEvent.query_id = event.getQueryId(); + thriftEvent.transaction_id = event.getTransactionId(); + thriftEvent.user = event.getUser(); + thriftEvent.principal = event.getPrincipal(); + thriftEvent.source = event.getSource(); + thriftEvent.server_version = event.getServerVersion(); + thriftEvent.environment = event.getEnvironment(); + thriftEvent.catalog = event.getCatalog(); + thriftEvent.schema = event.getSchema(); + thriftEvent.remote_client_address = event.getRemoteClientAddress(); + thriftEvent.user_agent = event.getUserAgent(); + thriftEvent.query_state = QueryState.valueOf(event.getQueryState()); + thriftEvent.uri = event.getUri(); + thriftEvent.field_names = event.getFieldNames(); + thriftEvent.query = event.getQuery(); + thriftEvent.create_time_ms = event.getCreateTime().getMillis(); + thriftEvent.execution_start_time_ms = event.getExecutionStartTime().getMillis(); + thriftEvent.end_time_ms = event.getEndTime().getMillis(); + thriftEvent.queued_time_ms = event.getQueuedTimeMs(); + if (event.getAnalysisTimeMs() != null) { + thriftEvent.analysis_time_ms = event.getAnalysisTimeMs(); + } + if (event.getDistributedPlanningTimeMs() != null) { + thriftEvent.distributed_planning_time_ms = event.getDistributedPlanningTimeMs(); + } + if (event.getTotalSplitWallTimeMs() != null) { + thriftEvent.total_split_wall_time_ms = event.getTotalSplitWallTimeMs(); + } + if (event.getTotalSplitCpuTimeMs() != null) { + thriftEvent.total_split_cpu_time_ms = event.getTotalSplitCpuTimeMs(); + } + if (event.getTotalBytes() != null) { + thriftEvent.total_bytes = event.getTotalBytes(); + } + if (event.getTotalRows() != null) { + thriftEvent.total_rows = event.getTotalRows(); + } + thriftEvent.splits = event.getSplits(); + if (event.getErrorCode() != null) { + thriftEvent.error_code_id = event.getErrorCode(); + } + thriftEvent.error_code_name = event.getErrorCodeName(); + thriftEvent.failure_type = event.getFailureType(); + thriftEvent.failure_message = event.getFailureMessage(); + thriftEvent.failure_task = event.getFailureTask(); + thriftEvent.failure_host = event.getFailureHost(); + thriftEvent.output_stage_json = event.getOutputStageJson(); + thriftEvent.failures_json = event.getFailuresJson(); + thriftEvent.inputs_json = event.getInputsJson(); + thriftEvent.session_properties_json = event.getSessionPropertiesJson(); + thriftEvent.query_wall_time_ms = event.getQueryWallTimeMs(); + if (event.getBytesPerSec() != null) { + thriftEvent.bytes_per_sec = event.getBytesPerSec(); + } + if (event.getBytesPerCpuSec() != null) { + thriftEvent.bytes_per_cpu_sec = event.getBytesPerCpuSec(); + } + if (event.getRowsPerSec() != null) { + thriftEvent.rows_per_sec = event.getRowsPerSec(); + } + if (event.getRowsPerCpuSec() != null) { + thriftEvent.rows_per_cpu_sec = event.getRowsPerCpuSec(); + } + + return thriftEvent; + } +} From 38e6d09f7a30f27c21b2402bd5f0895ffbf08199 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Tue, 29 Mar 2016 13:39:30 -0700 Subject: [PATCH 069/331] code clean-up --- .../facebook/presto/event/EventProcessor.java | 3 +++ .../facebook/presto/event/query/QueryEvent.java | 3 --- .../presto/twitter/logging/QueryScriber.java | 17 ++++++++--------- 3 files changed, 11 insertions(+), 12 deletions(-) diff --git a/presto-main/src/main/java/com/facebook/presto/event/EventProcessor.java b/presto-main/src/main/java/com/facebook/presto/event/EventProcessor.java index edaabc5437de..1ac55ea6d8a7 100644 --- a/presto-main/src/main/java/com/facebook/presto/event/EventProcessor.java +++ b/presto-main/src/main/java/com/facebook/presto/event/EventProcessor.java @@ -27,6 +27,9 @@ import java.io.IOException; import java.util.Set; +/** + * Class that listens for airlift events and sends presto events to handlers + */ public class EventProcessor extends AbstractEventClient { private static final String QUERY_CREATED = "QueryCreated"; diff --git a/presto-main/src/main/java/com/facebook/presto/event/query/QueryEvent.java b/presto-main/src/main/java/com/facebook/presto/event/query/QueryEvent.java index 33ca974b826c..da43e387ee22 100644 --- a/presto-main/src/main/java/com/facebook/presto/event/query/QueryEvent.java +++ b/presto-main/src/main/java/com/facebook/presto/event/query/QueryEvent.java @@ -13,9 +13,6 @@ */ package com.facebook.presto.event.query; -/** - * - */ public interface QueryEvent { String getQueryId(); diff --git a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java index 0110afdf8a61..d4450aaf6a38 100644 --- a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java +++ b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java @@ -13,13 +13,13 @@ */ package com.facebook.presto.twitter.logging; -import com.facebook.presto.event.query.QueryCompletionEvent; import com.facebook.presto.event.query.QueryEventHandler; import com.google.common.base.Optional; import com.twitter.logging.BareFormatter$; import com.twitter.logging.Level; import com.twitter.logging.QueueingHandler; import com.twitter.logging.ScribeHandler; +import com.twitter.presto.thriftjava.QueryCompletionEvent; import com.twitter.presto.thriftjava.QueryState; import io.airlift.log.Logger; import org.apache.thrift.TBase; @@ -32,7 +32,7 @@ /** * Class that scribes query completion events */ -public class QueryScriber implements QueryEventHandler +public class QueryScriber implements QueryEventHandler { private static final String SCRIBE_CATEGORY = "test_presto_query_complete"; private static final int MAX_QUEUE_SIZE = 1000; @@ -66,17 +66,17 @@ public QueryScriber() } @Override - public void handle(QueryCompletionEvent event) + public void handle(com.facebook.presto.event.query.QueryCompletionEvent event) { com.twitter.presto.thriftjava.QueryCompletionEvent thriftEvent = toThriftQueryCompletionEvent(event); Optional message = serializeThriftToString(thriftEvent); if (message.isPresent()) { - LogRecord logRecord = new LogRecord(Level.ALL, message.get()); - queueingHandler.publish(logRecord); + LogRecord logRecord = new LogRecord(Level.ALL, message.get()); + queueingHandler.publish(logRecord); } else { - log.warn("Unable to serialize QueryCompletionEvent: " + event); + log.warn("Unable to serialize QueryCompletionEvent: " + event); } } @@ -95,10 +95,9 @@ private Optional serializeThriftToString(TBase thriftMessage) } } - private static com.twitter.presto.thriftjava.QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletionEvent event) + private static QueryCompletionEvent toThriftQueryCompletionEvent(com.facebook.presto.event.query.QueryCompletionEvent event) { - com.twitter.presto.thriftjava.QueryCompletionEvent thriftEvent = - new com.twitter.presto.thriftjava.QueryCompletionEvent(); + QueryCompletionEvent thriftEvent = new QueryCompletionEvent(); thriftEvent.query_id = event.getQueryId(); thriftEvent.transaction_id = event.getTransactionId(); From 2ac4c4225977b175d2ecab181aa1f4d9a4cb1907 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Tue, 29 Mar 2016 14:02:47 -0700 Subject: [PATCH 070/331] Fix package leakage --- .../main/java/com/facebook/presto/event/EventProcessor.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/presto-main/src/main/java/com/facebook/presto/event/EventProcessor.java b/presto-main/src/main/java/com/facebook/presto/event/EventProcessor.java index 1ac55ea6d8a7..c575a6a49818 100644 --- a/presto-main/src/main/java/com/facebook/presto/event/EventProcessor.java +++ b/presto-main/src/main/java/com/facebook/presto/event/EventProcessor.java @@ -18,7 +18,6 @@ import com.facebook.presto.event.query.QueryEvent; import com.facebook.presto.event.query.QueryEventHandler; import com.facebook.presto.event.query.SplitCompletionEvent; -import com.facebook.presto.twitter.logging.QueryLogger; import com.google.inject.Inject; import io.airlift.event.client.AbstractEventClient; import io.airlift.event.client.EventType; @@ -35,7 +34,7 @@ public class EventProcessor extends AbstractEventClient private static final String QUERY_CREATED = "QueryCreated"; private static final String QUERY_COMPLETION = "QueryCompletion"; private static final String SPLIT_COMPLETION = "SplitCompletion"; - private static final Logger log = Logger.get(QueryLogger.class); + private static final Logger log = Logger.get(EventProcessor.class); private Set> queryCreatedEventHandlers; private Set> queryCompletionEventHandlers; From a8d1ffc0bdadbd5bed81db42135f1ed2ea1479e5 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Tue, 29 Mar 2016 15:15:11 -0700 Subject: [PATCH 071/331] Move twitter classes to a twitter package --- .../com/facebook/presto/hive/BackgroundHiveSplitLoader.java | 2 +- .../java/com/facebook/presto/hive/HiveConnectorFactory.java | 3 +++ .../com/facebook/presto/hive/HivePageSourceProvider.java | 2 +- .../{ => twitter}/hive/MetastoreStaticClusterModule.java | 5 ++++- .../{ => twitter}/hive/MetastoreZkDiscoveryBasedModule.java | 3 ++- .../presto/{ => twitter}/hive/ZookeeperMetastoreMonitor.java | 2 +- .../{ => twitter}/hive/ZookeeperServersetHiveCluster.java | 4 +++- .../hive/ZookeeperServersetMetastoreConfig.java | 2 +- .../facebook/presto/{ => twitter}/hive/util/UgiUtils.java | 2 +- .../{ => twitter}/hive/TestZookeeperMetastoreMonitor.java | 4 ++-- .../hive/TestZookeeperServersetMetastoreConfig.java | 2 +- .../facebook/presto/{ => twitter}/hive/util/TestUtils.java | 2 +- 12 files changed, 21 insertions(+), 12 deletions(-) rename presto-hive/src/main/java/com/facebook/presto/{ => twitter}/hive/MetastoreStaticClusterModule.java (84%) rename presto-hive/src/main/java/com/facebook/presto/{ => twitter}/hive/MetastoreZkDiscoveryBasedModule.java (92%) rename presto-hive/src/main/java/com/facebook/presto/{ => twitter}/hive/ZookeeperMetastoreMonitor.java (99%) rename presto-hive/src/main/java/com/facebook/presto/{ => twitter}/hive/ZookeeperServersetHiveCluster.java (94%) rename presto-hive/src/main/java/com/facebook/presto/{ => twitter}/hive/ZookeeperServersetMetastoreConfig.java (98%) rename presto-hive/src/main/java/com/facebook/presto/{ => twitter}/hive/util/UgiUtils.java (97%) rename presto-hive/src/test/java/com/facebook/presto/{ => twitter}/hive/TestZookeeperMetastoreMonitor.java (98%) rename presto-hive/src/test/java/com/facebook/presto/{ => twitter}/hive/TestZookeeperServersetMetastoreConfig.java (98%) rename presto-hive/src/test/java/com/facebook/presto/{ => twitter}/hive/util/TestUtils.java (94%) 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 676b856e6f7c..cd8d88e0d9e8 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 @@ -16,10 +16,10 @@ import com.facebook.presto.hive.util.HiveFileIterator; import com.facebook.presto.hive.util.ResumableTask; import com.facebook.presto.hive.util.ResumableTasks; -import com.facebook.presto.hive.util.UgiUtils; import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.HostAddress; import com.facebook.presto.spi.predicate.TupleDomain; +import com.facebook.presto.twitter.hive.util.UgiUtils; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import io.airlift.units.DataSize; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveConnectorFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveConnectorFactory.java index f544a553bdbb..7aa27c3ea11f 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveConnectorFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveConnectorFactory.java @@ -29,6 +29,9 @@ import com.facebook.presto.spi.classloader.ThreadContextClassLoader; import com.facebook.presto.spi.security.ConnectorAccessControl; import com.facebook.presto.spi.type.TypeManager; +import com.facebook.presto.twitter.hive.MetastoreStaticClusterModule; +import com.facebook.presto.twitter.hive.MetastoreZkDiscoveryBasedModule; +import com.facebook.presto.twitter.hive.ZookeeperServersetMetastoreConfig; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableSet; import com.google.inject.Injector; 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 ff1c81f19367..1edee9f8d7d6 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 @@ -13,7 +13,6 @@ */ package com.facebook.presto.hive; -import com.facebook.presto.hive.util.UgiUtils; import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ConnectorPageSource; import com.facebook.presto.spi.ConnectorPageSourceProvider; @@ -23,6 +22,7 @@ import com.facebook.presto.spi.predicate.TupleDomain; import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.TypeManager; +import com.facebook.presto.twitter.hive.util.UgiUtils; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import org.apache.hadoop.conf.Configuration; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/MetastoreStaticClusterModule.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/MetastoreStaticClusterModule.java similarity index 84% rename from presto-hive/src/main/java/com/facebook/presto/hive/MetastoreStaticClusterModule.java rename to presto-hive/src/main/java/com/facebook/presto/twitter/hive/MetastoreStaticClusterModule.java index 18bfa3d1eff1..b8c1b7dc9a60 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/MetastoreStaticClusterModule.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/MetastoreStaticClusterModule.java @@ -11,8 +11,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.facebook.presto.hive; +package com.facebook.presto.twitter.hive; +import com.facebook.presto.hive.HiveCluster; +import com.facebook.presto.hive.StaticHiveCluster; +import com.facebook.presto.hive.StaticMetastoreConfig; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Scopes; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/MetastoreZkDiscoveryBasedModule.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/MetastoreZkDiscoveryBasedModule.java similarity index 92% rename from presto-hive/src/main/java/com/facebook/presto/hive/MetastoreZkDiscoveryBasedModule.java rename to presto-hive/src/main/java/com/facebook/presto/twitter/hive/MetastoreZkDiscoveryBasedModule.java index a4d84813ab76..775a5afaf4c8 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/MetastoreZkDiscoveryBasedModule.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/MetastoreZkDiscoveryBasedModule.java @@ -11,8 +11,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.facebook.presto.hive; +package com.facebook.presto.twitter.hive; +import com.facebook.presto.hive.HiveCluster; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Scopes; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperMetastoreMonitor.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperMetastoreMonitor.java similarity index 99% rename from presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperMetastoreMonitor.java rename to presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperMetastoreMonitor.java index 3789ca3c3271..e1d0f2011468 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperMetastoreMonitor.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperMetastoreMonitor.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.facebook.presto.hive; +package com.facebook.presto.twitter.hive; import com.google.common.net.HostAndPort; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetHiveCluster.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java similarity index 94% rename from presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetHiveCluster.java rename to presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java index 48f869ec8ed0..83642e41c50f 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetHiveCluster.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java @@ -11,8 +11,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.facebook.presto.hive; +package com.facebook.presto.twitter.hive; +import com.facebook.presto.hive.HiveCluster; +import com.facebook.presto.hive.HiveMetastoreClientFactory; import com.facebook.presto.hive.metastore.HiveMetastoreClient; import com.facebook.presto.spi.PrestoException; import com.google.common.net.HostAndPort; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetMetastoreConfig.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetMetastoreConfig.java similarity index 98% rename from presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetMetastoreConfig.java rename to presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetMetastoreConfig.java index 44c6f9d19188..26e36b469d0c 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/ZookeeperServersetMetastoreConfig.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetMetastoreConfig.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.facebook.presto.hive; +package com.facebook.presto.twitter.hive; import io.airlift.configuration.Config; import io.airlift.configuration.ConfigDescription; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/util/UgiUtils.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/UgiUtils.java similarity index 97% rename from presto-hive/src/main/java/com/facebook/presto/hive/util/UgiUtils.java rename to presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/UgiUtils.java index 08f2dde30ff2..6d540bbe3c5a 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/util/UgiUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/UgiUtils.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.facebook.presto.hive.util; +package com.facebook.presto.twitter.hive.util; import org.apache.hadoop.security.UserGroupInformation; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestZookeeperMetastoreMonitor.java b/presto-hive/src/test/java/com/facebook/presto/twitter/hive/TestZookeeperMetastoreMonitor.java similarity index 98% rename from presto-hive/src/test/java/com/facebook/presto/hive/TestZookeeperMetastoreMonitor.java rename to presto-hive/src/test/java/com/facebook/presto/twitter/hive/TestZookeeperMetastoreMonitor.java index 6b82efa22f97..d0eba9f4e6e5 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestZookeeperMetastoreMonitor.java +++ b/presto-hive/src/test/java/com/facebook/presto/twitter/hive/TestZookeeperMetastoreMonitor.java @@ -11,10 +11,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.facebook.presto.hive; +package com.facebook.presto.twitter.hive; import com.facebook.presto.hadoop.shaded.com.google.common.collect.ImmutableList; -import com.facebook.presto.hive.util.TestUtils; +import com.facebook.presto.twitter.hive.util.TestUtils; import com.google.common.net.HostAndPort; import io.airlift.log.Logger; import org.I0Itec.zkclient.ZkClient; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestZookeeperServersetMetastoreConfig.java b/presto-hive/src/test/java/com/facebook/presto/twitter/hive/TestZookeeperServersetMetastoreConfig.java similarity index 98% rename from presto-hive/src/test/java/com/facebook/presto/hive/TestZookeeperServersetMetastoreConfig.java rename to presto-hive/src/test/java/com/facebook/presto/twitter/hive/TestZookeeperServersetMetastoreConfig.java index bb1117697296..6992a752dea1 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestZookeeperServersetMetastoreConfig.java +++ b/presto-hive/src/test/java/com/facebook/presto/twitter/hive/TestZookeeperServersetMetastoreConfig.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.facebook.presto.hive; +package com.facebook.presto.twitter.hive; import com.google.common.collect.ImmutableMap; import org.testng.annotations.Test; diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/util/TestUtils.java b/presto-hive/src/test/java/com/facebook/presto/twitter/hive/util/TestUtils.java similarity index 94% rename from presto-hive/src/test/java/com/facebook/presto/hive/util/TestUtils.java rename to presto-hive/src/test/java/com/facebook/presto/twitter/hive/util/TestUtils.java index 4315bac667c2..379ad3877e32 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/util/TestUtils.java +++ b/presto-hive/src/test/java/com/facebook/presto/twitter/hive/util/TestUtils.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.facebook.presto.hive.util; +package com.facebook.presto.twitter.hive.util; import java.io.IOException; import java.net.ServerSocket; From 73c53029594b351bc9f593b657e6bd86c0475a82 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Tue, 29 Mar 2016 15:26:47 -0700 Subject: [PATCH 072/331] Default package is presto and thrift is custom --- .../facebook/presto/twitter/logging/QueryScriber.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java index d4450aaf6a38..306029550ed6 100644 --- a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java +++ b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java @@ -13,13 +13,13 @@ */ package com.facebook.presto.twitter.logging; +import com.facebook.presto.event.query.QueryCompletionEvent; import com.facebook.presto.event.query.QueryEventHandler; import com.google.common.base.Optional; import com.twitter.logging.BareFormatter$; import com.twitter.logging.Level; import com.twitter.logging.QueueingHandler; import com.twitter.logging.ScribeHandler; -import com.twitter.presto.thriftjava.QueryCompletionEvent; import com.twitter.presto.thriftjava.QueryState; import io.airlift.log.Logger; import org.apache.thrift.TBase; @@ -32,7 +32,7 @@ /** * Class that scribes query completion events */ -public class QueryScriber implements QueryEventHandler +public class QueryScriber implements QueryEventHandler { private static final String SCRIBE_CATEGORY = "test_presto_query_complete"; private static final int MAX_QUEUE_SIZE = 1000; @@ -66,7 +66,7 @@ public QueryScriber() } @Override - public void handle(com.facebook.presto.event.query.QueryCompletionEvent event) + public void handle(QueryCompletionEvent event) { com.twitter.presto.thriftjava.QueryCompletionEvent thriftEvent = toThriftQueryCompletionEvent(event); Optional message = serializeThriftToString(thriftEvent); @@ -95,9 +95,10 @@ private Optional serializeThriftToString(TBase thriftMessage) } } - private static QueryCompletionEvent toThriftQueryCompletionEvent(com.facebook.presto.event.query.QueryCompletionEvent event) + private static com.twitter.presto.thriftjava.QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletionEvent event) { - QueryCompletionEvent thriftEvent = new QueryCompletionEvent(); + com.twitter.presto.thriftjava.QueryCompletionEvent thriftEvent = + new com.twitter.presto.thriftjava.QueryCompletionEvent(); thriftEvent.query_id = event.getQueryId(); thriftEvent.transaction_id = event.getTransactionId(); From fadd0f13f702a516a67b29c82dc8880ef19fce42 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Wed, 30 Mar 2016 10:08:39 -0700 Subject: [PATCH 073/331] Removing test from scribe category --- .../java/com/facebook/presto/twitter/logging/QueryScriber.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java index 306029550ed6..ad3e52fb9c21 100644 --- a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java +++ b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java @@ -34,7 +34,7 @@ */ public class QueryScriber implements QueryEventHandler { - private static final String SCRIBE_CATEGORY = "test_presto_query_complete"; + private static final String SCRIBE_CATEGORY = "presto_query_complete"; private static final int MAX_QUEUE_SIZE = 1000; private static final Logger log = Logger.get(QueryScriber.class); From 2e5c3f2ee54859b6f8d9a3a4c14c20da836175fd Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Wed, 30 Mar 2016 10:25:12 -0700 Subject: [PATCH 074/331] change version to avoid merge conflicts --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 37 files changed, 38 insertions(+), 38 deletions(-) diff --git a/pom.xml b/pom.xml index ae36b8cf3d9c..dbe9ff96aecd 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.141-tw-0.20 + 0.143 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 00cc629d364c..f97c8bba0c51 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 3652e17bd06f..428c8c649476 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 8f86dabd3e25..2d6d529d0fae 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.141-tw-0.20 + 0.143 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 1c0d41086973..f939fcd80924 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 1bdeaecdcea2..b6558dd1ae17 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 4f0675eda340..fabc5079fb31 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index bdcff6cdb591..841119ce5d50 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index f54c94db8398..1afa4e2137b9 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 4b4ad310d3e1..0aa73f400de0 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 2c4cc361f6fb..e2cce02150c6 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 7bb7379f70b6..6321368c053f 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 09e13a19e68d..e327f50e7fd0 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 6e304d8b10bd..0b2e9211761b 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index cfa615c8d2dc..f6e99377daf2 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index cebd8614c068..2ec69e52029e 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 24ca263be84e..84c75bbc045b 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 4ae769912294..b9d05f4c16c9 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 31620ae9fe5a..2aa0870b3928 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index dfe5c5d08d7a..789340a2d1fc 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index b5e67298855a..1899f5fcca69 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 4722581cd3a5..35b56aa6b724 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index d710b58886f5..160e11c44e72 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 830af643de31..a9c126945d7a 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index c8e057ed14aa..018204dea0e0 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index bfe2383a567f..4b00570b97cd 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.141-tw-0.20 + 0.143 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 3b527cbaa5dc..dee555abf4bb 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 5c724e169c0c..e8b911e4b433 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 4c226d7edc39..8c618e94b57c 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 14038c648f0f..d4a6474c05bd 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 2c4b2c1f49eb..2a35482c59a9 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index a977f2d08420..39bbf912871a 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 301e4c4efe55..bb6947fe3db6 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index dc8b11426992..a1d92c2c2639 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 25c2414ce83f..a81e47fb1e3e 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.141-tw-0.20 + 0.143 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 3979e5a18555..62667b35f11d 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 9dbeee8b8252..21c117a922ad 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.141-tw-0.20 + 0.143 presto-verifier From 1673479f824df2bfe0c63c91233eb1709d08dad7 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Wed, 30 Mar 2016 10:32:32 -0700 Subject: [PATCH 075/331] Update version to 0.143-tw-21 --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 37 files changed, 38 insertions(+), 38 deletions(-) diff --git a/pom.xml b/pom.xml index a15ff943598a..4d3e280a8b86 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.143 + 0.143-tw-21 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index f97c8bba0c51..9a3b30321ab3 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 428c8c649476..3e9996b54c4a 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 2d6d529d0fae..f678a84674f6 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.143 + 0.143-tw-21 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index f939fcd80924..96374f2176cf 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index b6558dd1ae17..bb5f42df5816 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index d75efa6ae5d0..2d0b7b238afd 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 841119ce5d50..b1b1c9af3a69 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 1afa4e2137b9..89ffee3b5137 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 0aa73f400de0..46f1cd60f31f 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index e2cce02150c6..ba4b98f37e6a 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index d6249e4c7649..dfab48e8c1fe 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 45dafbefb8e4..a7ed3e068632 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 5dd2f1cdb7b1..d1888fde6fe7 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 24e1bd0aa0d1..3f6eae9a0306 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 0ad2ec1f4a09..f1551983ce7c 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 84c75bbc045b..5840a8d8c5db 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index b9d05f4c16c9..e29367f1e07d 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 2aa0870b3928..b19205980eee 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 789340a2d1fc..955f03a0ec5b 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 1899f5fcca69..04c5a023509f 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 35b56aa6b724..9aecc0f8e037 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 160e11c44e72..b132a24e22e8 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index a9c126945d7a..2af8ad644181 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 018204dea0e0..0929844e2231 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 6a0ad32d8c94..7f8944a0ecbd 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.143 + 0.143-tw-21 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index dee555abf4bb..067fef0d7288 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index e8b911e4b433..b9cb797cbbc1 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 8c618e94b57c..10149cb92d1b 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index d4a6474c05bd..38e7af2f19cc 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 2a35482c59a9..9183920dfa3c 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 39bbf912871a..c6f3399b8ff7 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index bb6947fe3db6..81dc9466c696 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index a1d92c2c2639..ea738c49db14 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index a81e47fb1e3e..4376173271e8 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.143 + 0.143-tw-21 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 62667b35f11d..9a704e9671b1 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 21c117a922ad..3511883027ed 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143 + 0.143-tw-21 presto-verifier From 4e48cbd905adb085c1986050632beb2679796c63 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Wed, 30 Mar 2016 11:07:23 -0700 Subject: [PATCH 076/331] Fixed to not use illegal import --- .../presto/twitter/hive/TestZookeeperMetastoreMonitor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-hive/src/test/java/com/facebook/presto/twitter/hive/TestZookeeperMetastoreMonitor.java b/presto-hive/src/test/java/com/facebook/presto/twitter/hive/TestZookeeperMetastoreMonitor.java index d0eba9f4e6e5..1f89464aedce 100644 --- a/presto-hive/src/test/java/com/facebook/presto/twitter/hive/TestZookeeperMetastoreMonitor.java +++ b/presto-hive/src/test/java/com/facebook/presto/twitter/hive/TestZookeeperMetastoreMonitor.java @@ -13,8 +13,8 @@ */ package com.facebook.presto.twitter.hive; -import com.facebook.presto.hadoop.shaded.com.google.common.collect.ImmutableList; import com.facebook.presto.twitter.hive.util.TestUtils; +import com.google.common.collect.ImmutableList; import com.google.common.net.HostAndPort; import io.airlift.log.Logger; import org.I0Itec.zkclient.ZkClient; From 55a39606d49d3706b98dc1fa20285dec76614b7c Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Wed, 30 Mar 2016 11:22:11 -0700 Subject: [PATCH 077/331] Reduce logging of unsupported HttpRequest events --- .../src/main/java/com/facebook/presto/event/EventProcessor.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/presto-main/src/main/java/com/facebook/presto/event/EventProcessor.java b/presto-main/src/main/java/com/facebook/presto/event/EventProcessor.java index c575a6a49818..eef91027a4a1 100644 --- a/presto-main/src/main/java/com/facebook/presto/event/EventProcessor.java +++ b/presto-main/src/main/java/com/facebook/presto/event/EventProcessor.java @@ -72,8 +72,6 @@ protected void postEvent(T event) case SPLIT_COMPLETION: handle(splitCompletionEventHandlers, type, (SplitCompletionEvent) event); break; - default: - log.warn("Unrecognized event found: " + type); } } From 8729a4ef5268876deef4c1ebc36630ddb9607fe7 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Wed, 30 Mar 2016 13:17:31 -0700 Subject: [PATCH 078/331] Fix version from 0.143-tw-21 to 0.143-tw-0.21 --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 37 files changed, 38 insertions(+), 38 deletions(-) diff --git a/pom.xml b/pom.xml index 4d3e280a8b86..dcd36597ae72 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.143-tw-21 + 0.143-tw-0.21 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 9a3b30321ab3..f5509ae7ace0 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 3e9996b54c4a..f4798cbed1db 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index f678a84674f6..9cdb2ba7a288 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.143-tw-21 + 0.143-tw-0.21 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 96374f2176cf..cee636f28457 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index bb5f42df5816..3f9d4cc315b8 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 2d0b7b238afd..1584fab21a1a 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index b1b1c9af3a69..0112a2ec8df3 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 89ffee3b5137..86694f5e8351 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 46f1cd60f31f..802a03d546d3 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index ba4b98f37e6a..93ecc58122af 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index dfab48e8c1fe..98bf09737d7f 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index a7ed3e068632..5a5325b78ffd 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index d1888fde6fe7..c497b8c804e2 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 3f6eae9a0306..08a42164f5a9 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index f1551983ce7c..f2b6bfe18024 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 5840a8d8c5db..0649fdbca865 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index e29367f1e07d..6e482242ae9e 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index b19205980eee..5bb58fec8ffc 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 955f03a0ec5b..51a6e06ebe65 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 04c5a023509f..c017ed0e635d 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 9aecc0f8e037..a85b0b403e1f 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index b132a24e22e8..a67f058a17ed 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 2af8ad644181..97e0abd530f1 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 0929844e2231..7b2409dcb2ea 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 7f8944a0ecbd..b14447d4255d 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.143-tw-21 + 0.143-tw-0.21 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 067fef0d7288..98098e7bca6b 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index b9cb797cbbc1..007a962d5113 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 10149cb92d1b..d0e5fcb1a355 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 38e7af2f19cc..8c699831a585 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 9183920dfa3c..0c4200ec3f96 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index c6f3399b8ff7..26814c291cf9 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 81dc9466c696..9d0629fca8e8 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index ea738c49db14..fca9e2640ba5 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 4376173271e8..4ab85400cfe8 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.143-tw-21 + 0.143-tw-0.21 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 9a704e9671b1..38342b08d9b7 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 3511883027ed..2d075a28a214 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-21 + 0.143-tw-0.21 presto-verifier From e6579abad2f31e5966c5edffda2ca19ed3deb5a2 Mon Sep 17 00:00:00 2001 From: Bill Graham Date: Wed, 30 Mar 2016 22:40:07 -0700 Subject: [PATCH 079/331] Fix name of log category --- .../java/com/facebook/presto/twitter/logging/QueryScriber.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java index ad3e52fb9c21..5e55aa1ac195 100644 --- a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java +++ b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java @@ -34,7 +34,7 @@ */ public class QueryScriber implements QueryEventHandler { - private static final String SCRIBE_CATEGORY = "presto_query_complete"; + private static final String SCRIBE_CATEGORY = "presto_query_completion"; private static final int MAX_QUEUE_SIZE = 1000; private static final Logger log = Logger.get(QueryScriber.class); From a1ba15bbfbadc8c46d5cc2118b8a53c9e5365455 Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Thu, 21 Jul 2016 17:38:02 -0700 Subject: [PATCH 080/331] Fix scala dependency issues with twitter-master; removed scala 2.11 dependencies from thrift-java; downgraded util-logging to 2.10 --- pom.xml | 19 ++++++++++++++++--- presto-main/pom.xml | 2 +- 2 files changed, 17 insertions(+), 4 deletions(-) diff --git a/pom.xml b/pom.xml index dcd36597ae72..7998b80610f2 100644 --- a/pom.xml +++ b/pom.xml @@ -768,12 +768,25 @@ org.scala-lang.modules scala-parser-combinators_2.11 + + com.twitter + scrooge-core + + + org.scala-lang + scala-library + + + org.scala-lang + scala-reflect + + com.twitter - util-logging_2.11 - 6.33.0 + util-logging_2.10 + 6.34.0 commons-logging @@ -784,7 +797,7 @@ org.scala-lang scala-library - 2.11.7 + 2.10.6 commons-logging diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 51a6e06ebe65..f7aeef580f94 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -304,7 +304,7 @@ com.twitter - util-logging_2.11 + util-logging_2.10 org.apache.thrift From 484459328f2161d2e31d34112d0105cda72ec342 Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Tue, 9 Aug 2016 11:42:56 -0700 Subject: [PATCH 081/331] Cherry pick Tableau connector fix with bad defaults --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- .../src/main/resources/webapp/tableau/presto-client.js | 10 ++++++++-- .../resources/webapp/tableau/presto-connector.html | 6 ++---- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 39 files changed, 48 insertions(+), 44 deletions(-) diff --git a/pom.xml b/pom.xml index 7998b80610f2..f1e99d883be1 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.143-tw-0.21 + 0.143-tw-0.22 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index f5509ae7ace0..f723feaac0d2 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index f4798cbed1db..f2b8bbbc0f79 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 9cdb2ba7a288..7a40052b3460 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.143-tw-0.21 + 0.143-tw-0.22 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index cee636f28457..5127b74af96c 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 3f9d4cc315b8..8ecc331b8526 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 1584fab21a1a..cbf7b6065d3d 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 0112a2ec8df3..7a808d8896b8 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 86694f5e8351..09d5d5d01fbb 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 802a03d546d3..d293241818a3 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 93ecc58122af..59bd652ed59d 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 98bf09737d7f..b10f1b9646fe 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 5a5325b78ffd..1e0c40197d7a 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index c497b8c804e2..f9f34bfbe92e 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 08a42164f5a9..04cc2752f767 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index f2b6bfe18024..7602d1fe2634 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 0649fdbca865..36bfcdf41432 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 6e482242ae9e..e23c28c1df56 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 5bb58fec8ffc..7bced3cb454b 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index f7aeef580f94..6f9499075e9f 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-main diff --git a/presto-main/src/main/resources/webapp/tableau/presto-client.js b/presto-main/src/main/resources/webapp/tableau/presto-client.js index e0f0a14f4ebc..3bc357dd1b65 100644 --- a/presto-main/src/main/resources/webapp/tableau/presto-client.js +++ b/presto-main/src/main/resources/webapp/tableau/presto-client.js @@ -28,11 +28,17 @@ function StatementClient(connectionData, headerCallback, dataCallback, errorCall this.headers = { "X-Presto-User": this.user ? this.user : 'N/A', "X-Presto-Source": this.source, - "X-Presto-Catalog": this.catalog, - "X-Presto-Schema": this.schema, "X-Presto-Session": this.sessionParameters }; + if (!(this.catalog === undefined)) { + this.headers["X-Presto-Catalog"] = this.catalog + } + + if (!(this.schema === undefined)) { + this.headers["X-Presto-Schema"] = this.schema + } + // lastRecordNumber starts with 0 according to Tableau web connector docs this.submitQuery(0); } diff --git a/presto-main/src/main/resources/webapp/tableau/presto-connector.html b/presto-main/src/main/resources/webapp/tableau/presto-connector.html index 1b00dd795845..3b915fdfb4b2 100644 --- a/presto-main/src/main/resources/webapp/tableau/presto-connector.html +++ b/presto-main/src/main/resources/webapp/tableau/presto-connector.html @@ -135,8 +135,6 @@ function populateCatalogs() { var catalogClient = new StatementClient({ - 'catalog': 'hive', - 'schema': 'default', 'query': 'show catalogs' }, function() {}, @@ -191,8 +189,8 @@ } populateCatalogs(); - // initially load the schemas of the hive catalog - populateSchemas('hive'); + // initially load the schemas of the first catalog + populateSchemas($('#catalog option:selected').val()); document.getElementById("catalog").addEventListener('change', function() { diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index c017ed0e635d..e7283f16351e 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index a85b0b403e1f..9f01fee99131 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index a67f058a17ed..fdd0ceab14f0 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 97e0abd530f1..817ffe5fc72a 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 7b2409dcb2ea..910721d13e73 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index b14447d4255d..b558349d3f8f 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.143-tw-0.21 + 0.143-tw-0.22 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 98098e7bca6b..dc6480483553 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 007a962d5113..1d6638c663f7 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index d0e5fcb1a355..d53c3390fa7f 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 8c699831a585..653ed487fc2e 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 0c4200ec3f96..117ed5996b80 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 26814c291cf9..e424ee3d0725 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 9d0629fca8e8..de117022e736 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index fca9e2640ba5..5dc0a25029fd 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 4ab85400cfe8..8a66e9e37e43 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.143-tw-0.21 + 0.143-tw-0.22 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 38342b08d9b7..d861a447aafc 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 2d075a28a214..63bf1e044ee3 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-verifier From 1df4359c90121bd58773d9c77b2320e728aab130 Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Mon, 15 Aug 2016 11:17:27 -0700 Subject: [PATCH 082/331] Cherry pick Tableau connector fix with bad defaults --- .../src/main/resources/webapp/tableau/presto-client.js | 10 ++++++++-- .../resources/webapp/tableau/presto-connector.html | 6 ++---- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/presto-main/src/main/resources/webapp/tableau/presto-client.js b/presto-main/src/main/resources/webapp/tableau/presto-client.js index e0f0a14f4ebc..3bc357dd1b65 100644 --- a/presto-main/src/main/resources/webapp/tableau/presto-client.js +++ b/presto-main/src/main/resources/webapp/tableau/presto-client.js @@ -28,11 +28,17 @@ function StatementClient(connectionData, headerCallback, dataCallback, errorCall this.headers = { "X-Presto-User": this.user ? this.user : 'N/A', "X-Presto-Source": this.source, - "X-Presto-Catalog": this.catalog, - "X-Presto-Schema": this.schema, "X-Presto-Session": this.sessionParameters }; + if (!(this.catalog === undefined)) { + this.headers["X-Presto-Catalog"] = this.catalog + } + + if (!(this.schema === undefined)) { + this.headers["X-Presto-Schema"] = this.schema + } + // lastRecordNumber starts with 0 according to Tableau web connector docs this.submitQuery(0); } diff --git a/presto-main/src/main/resources/webapp/tableau/presto-connector.html b/presto-main/src/main/resources/webapp/tableau/presto-connector.html index 1b00dd795845..3b915fdfb4b2 100644 --- a/presto-main/src/main/resources/webapp/tableau/presto-connector.html +++ b/presto-main/src/main/resources/webapp/tableau/presto-connector.html @@ -135,8 +135,6 @@ function populateCatalogs() { var catalogClient = new StatementClient({ - 'catalog': 'hive', - 'schema': 'default', 'query': 'show catalogs' }, function() {}, @@ -191,8 +189,8 @@ } populateCatalogs(); - // initially load the schemas of the hive catalog - populateSchemas('hive'); + // initially load the schemas of the first catalog + populateSchemas($('#catalog option:selected').val()); document.getElementById("catalog").addEventListener('change', function() { From 1e07aeaa7fee3a719f0875ff853c1ae495cd9d33 Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Tue, 9 Aug 2016 11:42:56 -0700 Subject: [PATCH 083/331] Cherry pick Tableau connector fix with bad defaults --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 37 files changed, 38 insertions(+), 38 deletions(-) diff --git a/pom.xml b/pom.xml index 7998b80610f2..f1e99d883be1 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.143-tw-0.21 + 0.143-tw-0.22 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index f5509ae7ace0..f723feaac0d2 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index f4798cbed1db..f2b8bbbc0f79 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 9cdb2ba7a288..7a40052b3460 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.143-tw-0.21 + 0.143-tw-0.22 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index cee636f28457..5127b74af96c 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 3f9d4cc315b8..8ecc331b8526 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 1584fab21a1a..cbf7b6065d3d 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 0112a2ec8df3..7a808d8896b8 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 86694f5e8351..09d5d5d01fbb 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 802a03d546d3..d293241818a3 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 93ecc58122af..59bd652ed59d 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 98bf09737d7f..b10f1b9646fe 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 5a5325b78ffd..1e0c40197d7a 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index c497b8c804e2..f9f34bfbe92e 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 08a42164f5a9..04cc2752f767 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index f2b6bfe18024..7602d1fe2634 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 0649fdbca865..36bfcdf41432 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 6e482242ae9e..e23c28c1df56 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 5bb58fec8ffc..7bced3cb454b 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index f7aeef580f94..6f9499075e9f 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index c017ed0e635d..e7283f16351e 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index a85b0b403e1f..9f01fee99131 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index a67f058a17ed..fdd0ceab14f0 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 97e0abd530f1..817ffe5fc72a 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 7b2409dcb2ea..910721d13e73 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index b14447d4255d..b558349d3f8f 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.143-tw-0.21 + 0.143-tw-0.22 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 98098e7bca6b..dc6480483553 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 007a962d5113..1d6638c663f7 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index d0e5fcb1a355..d53c3390fa7f 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 8c699831a585..653ed487fc2e 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 0c4200ec3f96..117ed5996b80 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 26814c291cf9..e424ee3d0725 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 9d0629fca8e8..de117022e736 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index fca9e2640ba5..5dc0a25029fd 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 4ab85400cfe8..8a66e9e37e43 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.143-tw-0.21 + 0.143-tw-0.22 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 38342b08d9b7..d861a447aafc 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 2d075a28a214..63bf1e044ee3 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.21 + 0.143-tw-0.22 presto-verifier From 6e9bfea61ae214640c764d57da42aeaefced1387 Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Thu, 18 Aug 2016 11:46:43 -0700 Subject: [PATCH 084/331] add hack to javascript to make requests in https if page was visited in https --- .../src/main/resources/webapp/tableau/presto-client.js | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/presto-main/src/main/resources/webapp/tableau/presto-client.js b/presto-main/src/main/resources/webapp/tableau/presto-client.js index 3bc357dd1b65..abd6c8b045e9 100644 --- a/presto-main/src/main/resources/webapp/tableau/presto-client.js +++ b/presto-main/src/main/resources/webapp/tableau/presto-client.js @@ -17,6 +17,8 @@ function StatementClient(connectionData, headerCallback, dataCallback, errorCall this.currentResults = null; this.valid = true; + this.isHttps = window.location.protocol === "https:" + if (!(connectionData.sessionParameters === undefined)) { var parameterMap = JSON.parse(connectionData.sessionParameters); for (var name in parameterMap) { @@ -72,7 +74,7 @@ StatementClient.prototype.advance = function(lastRecordNumber) { var statementClient = this; $.ajax({ type: "GET", - url: this.currentResults.nextUri, + url: this.isHttps ? this.currentResults.nextUri.replace("http", "https") : this.currentResults.nextUri, headers: this.headers, dataType: 'json', // FIXME having problems when async: true From bf4dec27e18098a0a8c29d5cbdba57c84cb3b6df Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Thu, 18 Aug 2016 13:14:08 -0700 Subject: [PATCH 085/331] address Zhihao's comment to mind case where nextUri might already have https --- presto-main/src/main/resources/webapp/tableau/presto-client.js | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-main/src/main/resources/webapp/tableau/presto-client.js b/presto-main/src/main/resources/webapp/tableau/presto-client.js index abd6c8b045e9..14de90193895 100644 --- a/presto-main/src/main/resources/webapp/tableau/presto-client.js +++ b/presto-main/src/main/resources/webapp/tableau/presto-client.js @@ -74,7 +74,7 @@ StatementClient.prototype.advance = function(lastRecordNumber) { var statementClient = this; $.ajax({ type: "GET", - url: this.isHttps ? this.currentResults.nextUri.replace("http", "https") : this.currentResults.nextUri, + url: this.isHttps ? this.currentResults.nextUri.replace(/^http:/, 'https:') : this.currentResults.nextUri, headers: this.headers, dataType: 'json', // FIXME having problems when async: true From 443f76b73ef3131e7383656d86ae5721cac7ca4e Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Thu, 18 Aug 2016 14:41:16 -0700 Subject: [PATCH 086/331] update tag to 0.143-tw-0.23 --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 37 files changed, 38 insertions(+), 38 deletions(-) diff --git a/pom.xml b/pom.xml index f1e99d883be1..56888bf9ec0e 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.143-tw-0.22 + 0.143-tw-0.23 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index f723feaac0d2..91f462cfadb0 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index f2b8bbbc0f79..b3eb4d56e859 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 7a40052b3460..ca72cfe3d5d3 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.143-tw-0.22 + 0.143-tw-0.23 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 5127b74af96c..8aa14bce49fd 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 8ecc331b8526..7ade60a4cb0c 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index cbf7b6065d3d..990c3da0dba6 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 7a808d8896b8..0dbadefd62ae 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 09d5d5d01fbb..2d99c244cf0b 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index d293241818a3..e5db3aa84671 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 59bd652ed59d..08404aab6545 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index b10f1b9646fe..d9036e8f3d95 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 1e0c40197d7a..f9d45157f9fc 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index f9f34bfbe92e..ea710e04f016 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 04cc2752f767..f6f9554cc4f1 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 7602d1fe2634..deaf1b6f7ce2 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 36bfcdf41432..299cfbd382a3 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index e23c28c1df56..782d4fc3894f 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 7bced3cb454b..875a03aa63f7 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 6f9499075e9f..88d36739d70c 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index e7283f16351e..05a809b78044 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 9f01fee99131..19104889d0e1 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index fdd0ceab14f0..ff25cd66c973 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 817ffe5fc72a..0040747083d3 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 910721d13e73..c0d7e1b5c67c 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index b558349d3f8f..d3c75dc22d20 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.143-tw-0.22 + 0.143-tw-0.23 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index dc6480483553..f6df5f755d74 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 1d6638c663f7..85e013e3bc61 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index d53c3390fa7f..38200cc68020 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 653ed487fc2e..aa2792d3b0d6 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 117ed5996b80..9b4d72aa6ddf 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index e424ee3d0725..20dcbb999b82 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index de117022e736..9f0efb881244 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 5dc0a25029fd..11cf1b90fe7c 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 8a66e9e37e43..574825fa8a61 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.143-tw-0.22 + 0.143-tw-0.23 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index d861a447aafc..28dfc515bf06 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 63bf1e044ee3..430df7fc5da0 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.22 + 0.143-tw-0.23 presto-verifier From 67d22d7e4b8431fe3ea69734ceaad93e2cc0468e Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 25 Oct 2016 16:52:05 -0700 Subject: [PATCH 087/331] Added shuffle for metastores in order to balance connection load --- .../presto/twitter/hive/ZookeeperServersetHiveCluster.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java index 83642e41c50f..1885f515848d 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java @@ -23,6 +23,7 @@ import javax.inject.Inject; +import java.util.Collections; import java.util.List; import static com.facebook.presto.hive.HiveErrorCode.HIVE_METASTORE_ERROR; @@ -51,6 +52,7 @@ public ZookeeperServersetHiveCluster(ZookeeperServersetMetastoreConfig config, H public HiveMetastoreClient createMetastoreClient() { List metastores = zkMetastoreMonitor.getServers(); + Collections.shuffle(metastores); TTransportException lastException = null; for (HostAndPort metastore : metastores) { try { From ecfa6c0a656f3f59864fd5f49efd1e7c84220479 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 27 Oct 2016 17:49:54 -0700 Subject: [PATCH 088/331] Added log for hive connection and log for RetryDriver --- .../src/main/java/com/facebook/presto/hive/RetryDriver.java | 4 +++- .../presto/twitter/hive/ZookeeperServersetHiveCluster.java | 3 ++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/RetryDriver.java b/presto-hive/src/main/java/com/facebook/presto/hive/RetryDriver.java index d969a522a750..c2c1fac24ffc 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/RetryDriver.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/RetryDriver.java @@ -136,16 +136,18 @@ public V run(String callableName, Callable callable) return callable.call(); } catch (Exception e) { + log.debug("Failed on executing %s with attempt %d, Exception: %s", callableName, attempt, e.getMessage()); e = exceptionMapper.apply(e); for (Class clazz : exceptionWhiteList) { if (clazz.isInstance(e)) { + log.debug("Exception is in whitelist."); throw e; } } if (attempt >= maxAttempts || Duration.nanosSince(startTime).compareTo(maxRetryTime) >= 0) { + log.debug("Maximum attempts or maximum retry time reached. attempt: %d, maxAttempts: %d, duration: [%s] maxRetryTime: [%s]", attempt, maxAttempts, Duration.nanosSince(startTime).toString(), maxRetryTime.toString()); throw e; } - log.debug("Failed on executing %s with attempt %d, will retry. Exception: %s", callableName, attempt, e.getMessage()); int delayInMs = (int) Math.min(minSleepTime.toMillis() * Math.pow(scaleFactor, attempt - 1), maxSleepTime.toMillis()); TimeUnit.MILLISECONDS.sleep(delayInMs); diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java index 1885f515848d..1f1b6cad01a4 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java @@ -60,10 +60,11 @@ public HiveMetastoreClient createMetastoreClient() return clientFactory.create(metastore.getHostText(), metastore.getPort()); } catch (TTransportException e) { + log.debug("Failed connecting to Hive metastore at: " + metastore.toString()); lastException = e; } } - throw new PrestoException(HIVE_METASTORE_ERROR, "Failed connecting to Hive metastore", lastException); + throw new PrestoException(HIVE_METASTORE_ERROR, "Failed connecting to Hive metastore: " + metastores, lastException); } } From dee633688c4aa2c49da0fa7c3549d22087eb54b1 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Mon, 31 Oct 2016 15:45:52 -0700 Subject: [PATCH 089/331] Used RuntimeException to replace PrestoException, prevent whitelist stops RetryDriver --- .../presto/twitter/hive/ZookeeperServersetHiveCluster.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java index 1f1b6cad01a4..ca684b269cf5 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java @@ -16,7 +16,6 @@ import com.facebook.presto.hive.HiveCluster; import com.facebook.presto.hive.HiveMetastoreClientFactory; import com.facebook.presto.hive.metastore.HiveMetastoreClient; -import com.facebook.presto.spi.PrestoException; import com.google.common.net.HostAndPort; import io.airlift.log.Logger; import org.apache.thrift.transport.TTransportException; @@ -26,7 +25,6 @@ import java.util.Collections; import java.util.List; -import static com.facebook.presto.hive.HiveErrorCode.HIVE_METASTORE_ERROR; import static java.util.Objects.requireNonNull; public class ZookeeperServersetHiveCluster @@ -65,6 +63,6 @@ public HiveMetastoreClient createMetastoreClient() } } - throw new PrestoException(HIVE_METASTORE_ERROR, "Failed connecting to Hive metastore: " + metastores, lastException); + throw new RuntimeException("Failed connecting to Hive metastore: " + metastores, lastException); } } From 56d7c362a1cdf4ec76d5b6975f8227fff853390a Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Mon, 31 Oct 2016 18:22:10 -0700 Subject: [PATCH 090/331] Used formatter for logs --- .../presto/twitter/hive/ZookeeperServersetHiveCluster.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java index ca684b269cf5..379352d4100e 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java @@ -54,15 +54,15 @@ public HiveMetastoreClient createMetastoreClient() TTransportException lastException = null; for (HostAndPort metastore : metastores) { try { - log.info("Connecting to metastore at: " + metastore.toString()); + log.info("Connecting to metastore at: %s", metastore.toString()); return clientFactory.create(metastore.getHostText(), metastore.getPort()); } catch (TTransportException e) { - log.debug("Failed connecting to Hive metastore at: " + metastore.toString()); + log.debug("Failed connecting to Hive metastore at: %s", metastore.toString()); lastException = e; } } - throw new RuntimeException("Failed connecting to Hive metastore: " + metastores, lastException); + throw new RuntimeException("Failed connecting to Hive metastore.", lastException); } } From 36175ec6d47d24ff3d7519f68d3e5c0d0af0fc98 Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Wed, 2 Nov 2016 15:19:05 -0700 Subject: [PATCH 091/331] remove obsolete event listener implementation - part 1 of 3 for upgrading to 0.155 --- pom.xml | 68 ------- presto-main/pom.xml | 18 -- .../facebook/presto/server/PrestoServer.java | 7 - .../presto/twitter/TwitterModuleLoader.java | 64 ------- .../presto/twitter/logging/QueryLogger.java | 78 -------- .../presto/twitter/logging/QueryScriber.java | 169 ------------------ 6 files changed, 404 deletions(-) delete mode 100644 presto-main/src/main/java/com/facebook/presto/twitter/TwitterModuleLoader.java delete mode 100644 presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java delete mode 100644 presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java diff --git a/pom.xml b/pom.xml index 56888bf9ec0e..73a2b815da39 100644 --- a/pom.xml +++ b/pom.xml @@ -737,74 +737,6 @@ hive-apache-jdbc 0.13.1-1 - - - - com.twitter - presto-thrift-java - 0.0.1 - - - com.twitter - util-core_2.11 - - - com.twitter - util-core-java - - - com.twitter - util-function_2.10 - - - com.twitter - util-function-java - - - commons-logging - commons-logging - - - org.scala-lang.modules - scala-parser-combinators_2.11 - - - com.twitter - scrooge-core - - - org.scala-lang - scala-library - - - org.scala-lang - scala-reflect - - - - - - com.twitter - util-logging_2.10 - 6.34.0 - - - commons-logging - commons-logging - - - - - org.scala-lang - scala-library - 2.10.6 - - - commons-logging - commons-logging - - - diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 88d36739d70c..c6f9b388ceab 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -296,24 +296,6 @@ tpch test - - - - com.twitter - presto-thrift-java - - - com.twitter - util-logging_2.10 - - - org.apache.thrift - libthrift - - - org.scala-lang - scala-library - diff --git a/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java b/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java index f7185a279dc7..fda5706d9ebe 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java +++ b/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java @@ -119,8 +119,6 @@ public void run() ); - modules.addAll(getAdditionalModules()); - Bootstrap app = new Bootstrap(modules.build()); try { @@ -149,11 +147,6 @@ public void run() } } - protected Iterable getAdditionalModules() - { - return com.facebook.presto.twitter.TwitterModuleLoader.getAdditionalModules(); - } - private static void updateDatasources(Announcer announcer, Metadata metadata, ServerConfig serverConfig, NodeSchedulerConfig schedulerConfig) { // get existing announcement diff --git a/presto-main/src/main/java/com/facebook/presto/twitter/TwitterModuleLoader.java b/presto-main/src/main/java/com/facebook/presto/twitter/TwitterModuleLoader.java deleted file mode 100644 index eb23a7650a09..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/twitter/TwitterModuleLoader.java +++ /dev/null @@ -1,64 +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.twitter; - -import com.facebook.presto.event.EventProcessor; -import com.facebook.presto.event.query.QueryCompletionEvent; -import com.facebook.presto.event.query.QueryCreatedEvent; -import com.facebook.presto.event.query.QueryEventHandler; -import com.facebook.presto.event.query.SplitCompletionEvent; -import com.facebook.presto.twitter.logging.QueryLogger; -import com.facebook.presto.twitter.logging.QueryScriber; -import com.google.common.collect.ImmutableList; -import com.google.inject.Module; -import com.google.inject.Scopes; -import com.google.inject.TypeLiteral; -import com.google.inject.multibindings.Multibinder; -import io.airlift.event.client.EventClient; - -/** - * Loader that initializes custom Twitter code to inject into Presto. Whenever - * possible we should use this pattern to inject custom functionality, since it - * makes it easier to differentiate our patches from the core OS code. - * - * If the functionality we wish to add/override isn't currently possible to via - * overriding a guice module, we should contribute the necessary modules/interfaces - * into the OS Presto code base to make it possible. - */ -public class TwitterModuleLoader -{ - private TwitterModuleLoader() - { - } - - public static Iterable getAdditionalModules() - { - return ImmutableList.of( - binder -> Multibinder.newSetBinder(binder, EventClient.class) - .addBinding() - .to(EventProcessor.class) - .in(Scopes.SINGLETON), - binder -> Multibinder.newSetBinder(binder, new TypeLiteral>() {}), - binder -> Multibinder.newSetBinder(binder, new TypeLiteral>() {}), - binder -> Multibinder.newSetBinder(binder, new TypeLiteral>(){}) - .addBinding() - .to(new TypeLiteral(){}) - .in(Scopes.SINGLETON), - binder -> Multibinder.newSetBinder(binder, new TypeLiteral>(){}) - .addBinding() - .to(QueryScriber.class) - .in(Scopes.SINGLETON) - ); - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java deleted file mode 100644 index 02edd8da2440..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java +++ /dev/null @@ -1,78 +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.twitter.logging; - -import com.facebook.presto.event.query.QueryCompletionEvent; -import com.facebook.presto.event.query.QueryEventHandler; -import com.facebook.presto.spi.StandardErrorCode; -import io.airlift.log.Logger; -import io.airlift.units.Duration; - -import java.util.concurrent.TimeUnit; - -/** - * Class that logs query events to a file - */ -public class QueryLogger implements QueryEventHandler -{ - private static final int MAX_QUERY_LENGTH = 1000; - private static final String DASH = "-"; - private static final String COLON = ":"; - private static final String SPACE = " "; - private static final String ELIPSIS = "..."; - private static final String QUERY_COMPLETION = "QueryCompletion"; - - private static final Logger log = Logger.get(QueryLogger.class); - - @Override - public void handle(QueryCompletionEvent event) - { - String errorType = DASH; - String errorCode = DASH; - if (event.getErrorCode() != null) { - errorType = StandardErrorCode.toErrorType(event.getErrorCode()).toString(); - if (event.getErrorCodeName() != null) { - errorCode = event.getErrorCodeName() + COLON + event.getErrorCode(); - } - } - - Duration duration = (new Duration( - event.getQueryWallTimeMs(), TimeUnit.MILLISECONDS)) - .convertToMostSuccinctTimeUnit(); - - log.info(String.format("%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s", - QUERY_COMPLETION, event.getQueryId(), toLogValue(event.getRemoteClientAddress()), - event.getQueryState(), errorType, errorCode, event.getUser(), duration, - event.getSplits(), event.getTotalRows(), event.getTotalBytes(), - cleanseAndTrimQuery(event.getQuery()))); - } - - private static String toLogValue(Object object) - { - if (object == null) { - return DASH; - } - else { - return object.toString(); - } - } - - private static String cleanseAndTrimQuery(String query) - { - if (query.length() > MAX_QUERY_LENGTH) { - query = query.substring(0, MAX_QUERY_LENGTH) + ELIPSIS; - } - return query.replace(System.getProperty("line.separator"), SPACE); - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java deleted file mode 100644 index 5e55aa1ac195..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java +++ /dev/null @@ -1,169 +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.twitter.logging; - -import com.facebook.presto.event.query.QueryCompletionEvent; -import com.facebook.presto.event.query.QueryEventHandler; -import com.google.common.base.Optional; -import com.twitter.logging.BareFormatter$; -import com.twitter.logging.Level; -import com.twitter.logging.QueueingHandler; -import com.twitter.logging.ScribeHandler; -import com.twitter.presto.thriftjava.QueryState; -import io.airlift.log.Logger; -import org.apache.thrift.TBase; -import org.apache.thrift.TException; -import org.apache.thrift.TSerializer; - -import java.util.Base64; -import java.util.logging.LogRecord; - -/** - * Class that scribes query completion events - */ -public class QueryScriber implements QueryEventHandler -{ - private static final String SCRIBE_CATEGORY = "presto_query_completion"; - private static final int MAX_QUEUE_SIZE = 1000; - - private static final Logger log = Logger.get(QueryScriber.class); - - private QueueingHandler queueingHandler; - - // TSerializer is not thread safe - private final ThreadLocal serializer = new ThreadLocal() - { - @Override protected TSerializer initialValue() - { - return new TSerializer(); - } - }; - - public QueryScriber() - { - ScribeHandler scribeHandler = new ScribeHandler( - ScribeHandler.DefaultHostname(), - ScribeHandler.DefaultPort(), - SCRIBE_CATEGORY, - ScribeHandler.DefaultBufferTime(), - ScribeHandler.DefaultConnectBackoff(), - ScribeHandler.DefaultMaxMessagesPerTransaction(), - ScribeHandler.DefaultMaxMessagesToBuffer(), - BareFormatter$.MODULE$, - scala.Option.apply((Level) null)); - queueingHandler = new QueueingHandler(scribeHandler, MAX_QUEUE_SIZE); - } - - @Override - public void handle(QueryCompletionEvent event) - { - com.twitter.presto.thriftjava.QueryCompletionEvent thriftEvent = toThriftQueryCompletionEvent(event); - Optional message = serializeThriftToString(thriftEvent); - - if (message.isPresent()) { - LogRecord logRecord = new LogRecord(Level.ALL, message.get()); - queueingHandler.publish(logRecord); - } - else { - log.warn("Unable to serialize QueryCompletionEvent: " + event); - } - } - - /** - * Serialize a thrift object to bytes, compress, then encode as a base64 string. - */ - private Optional serializeThriftToString(TBase thriftMessage) - { - try { - return Optional.of( - Base64.getEncoder().encodeToString(serializer.get().serialize(thriftMessage))); - } - catch (TException e) { - log.warn(e, "Could not serialize thrift object" + thriftMessage); - return Optional.absent(); - } - } - - private static com.twitter.presto.thriftjava.QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletionEvent event) - { - com.twitter.presto.thriftjava.QueryCompletionEvent thriftEvent = - new com.twitter.presto.thriftjava.QueryCompletionEvent(); - - thriftEvent.query_id = event.getQueryId(); - thriftEvent.transaction_id = event.getTransactionId(); - thriftEvent.user = event.getUser(); - thriftEvent.principal = event.getPrincipal(); - thriftEvent.source = event.getSource(); - thriftEvent.server_version = event.getServerVersion(); - thriftEvent.environment = event.getEnvironment(); - thriftEvent.catalog = event.getCatalog(); - thriftEvent.schema = event.getSchema(); - thriftEvent.remote_client_address = event.getRemoteClientAddress(); - thriftEvent.user_agent = event.getUserAgent(); - thriftEvent.query_state = QueryState.valueOf(event.getQueryState()); - thriftEvent.uri = event.getUri(); - thriftEvent.field_names = event.getFieldNames(); - thriftEvent.query = event.getQuery(); - thriftEvent.create_time_ms = event.getCreateTime().getMillis(); - thriftEvent.execution_start_time_ms = event.getExecutionStartTime().getMillis(); - thriftEvent.end_time_ms = event.getEndTime().getMillis(); - thriftEvent.queued_time_ms = event.getQueuedTimeMs(); - if (event.getAnalysisTimeMs() != null) { - thriftEvent.analysis_time_ms = event.getAnalysisTimeMs(); - } - if (event.getDistributedPlanningTimeMs() != null) { - thriftEvent.distributed_planning_time_ms = event.getDistributedPlanningTimeMs(); - } - if (event.getTotalSplitWallTimeMs() != null) { - thriftEvent.total_split_wall_time_ms = event.getTotalSplitWallTimeMs(); - } - if (event.getTotalSplitCpuTimeMs() != null) { - thriftEvent.total_split_cpu_time_ms = event.getTotalSplitCpuTimeMs(); - } - if (event.getTotalBytes() != null) { - thriftEvent.total_bytes = event.getTotalBytes(); - } - if (event.getTotalRows() != null) { - thriftEvent.total_rows = event.getTotalRows(); - } - thriftEvent.splits = event.getSplits(); - if (event.getErrorCode() != null) { - thriftEvent.error_code_id = event.getErrorCode(); - } - thriftEvent.error_code_name = event.getErrorCodeName(); - thriftEvent.failure_type = event.getFailureType(); - thriftEvent.failure_message = event.getFailureMessage(); - thriftEvent.failure_task = event.getFailureTask(); - thriftEvent.failure_host = event.getFailureHost(); - thriftEvent.output_stage_json = event.getOutputStageJson(); - thriftEvent.failures_json = event.getFailuresJson(); - thriftEvent.inputs_json = event.getInputsJson(); - thriftEvent.session_properties_json = event.getSessionPropertiesJson(); - thriftEvent.query_wall_time_ms = event.getQueryWallTimeMs(); - if (event.getBytesPerSec() != null) { - thriftEvent.bytes_per_sec = event.getBytesPerSec(); - } - if (event.getBytesPerCpuSec() != null) { - thriftEvent.bytes_per_cpu_sec = event.getBytesPerCpuSec(); - } - if (event.getRowsPerSec() != null) { - thriftEvent.rows_per_sec = event.getRowsPerSec(); - } - if (event.getRowsPerCpuSec() != null) { - thriftEvent.rows_per_cpu_sec = event.getRowsPerCpuSec(); - } - - return thriftEvent; - } -} From f89cc403a0d6705d8d354d1f309307e5d3091b03 Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Tue, 8 Nov 2016 11:34:20 -0800 Subject: [PATCH 092/331] update version first --- pom.xml | 4 ++-- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 37 files changed, 38 insertions(+), 38 deletions(-) diff --git a/pom.xml b/pom.xml index 73a2b815da39..c94af68e6215 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.143-tw-0.23 + 0.149 diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 91f462cfadb0..d5b8d1e78be0 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index b3eb4d56e859..c73d27de8a45 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index ca72cfe3d5d3..63198c198edc 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.143-tw-0.23 + 0.149 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 8aa14bce49fd..b03e8e6acc3c 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 7ade60a4cb0c..03d54389f987 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 990c3da0dba6..97f8ad0d6d2f 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 0dbadefd62ae..03c448212c15 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 2d99c244cf0b..58398e0dc6f5 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index e5db3aa84671..b8d82a0b8a7c 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 08404aab6545..8b83af250254 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index d9036e8f3d95..ea8070438a1d 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index f9d45157f9fc..72207865f38c 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index ea710e04f016..d305d142df4c 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index f6f9554cc4f1..e14863a1f68e 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index deaf1b6f7ce2..43ba4621569a 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 299cfbd382a3..fc710346266a 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 782d4fc3894f..d1a24c512bea 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 875a03aa63f7..142fcf2dac0c 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-kafka diff --git a/presto-main/pom.xml b/presto-main/pom.xml index c6f9b388ceab..64b6cbeffbf5 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 05a809b78044..1f28da25ceda 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-ml diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 19104889d0e1..d6e339e06e67 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index ff25cd66c973..77e592bab23d 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 0040747083d3..7a80dd13631b 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index c0d7e1b5c67c..b1095676e8d1 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index d3c75dc22d20..9502d7166a81 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.143-tw-0.23 + 0.149 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index f6df5f755d74..8a60522ac198 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 85e013e3bc61..fb3c6827e991 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 38200cc68020..fa0916fb9bf1 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index aa2792d3b0d6..f831831e0a5d 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 9b4d72aa6ddf..4ca0a9cf7397 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 20dcbb999b82..8cec5d288c1b 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 9f0efb881244..2a24fdc53a19 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 11cf1b90fe7c..c46298046c3b 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 574825fa8a61..2dc252ceb20a 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.143-tw-0.23 + 0.149 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 28dfc515bf06..d8c0c539d4e6 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 430df7fc5da0..dbfa75787fc6 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.143-tw-0.23 + 0.149 presto-verifier From 9b5d00094652c9354fb03e15f9353c6e1497c148 Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Tue, 8 Nov 2016 14:23:59 -0800 Subject: [PATCH 093/331] use new twitter version --- pom.xml | 4 ++-- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- .../com/facebook/presto/hive/BackgroundHiveSplitLoader.java | 4 ++-- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 41 files changed, 43 insertions(+), 43 deletions(-) diff --git a/pom.xml b/pom.xml index ae25ca9b6ca3..01e7b2e5eddf 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.149 + 0.149-tw-0.25 diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index d31f9c4d3504..85403124dc54 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index d5b8d1e78be0..dbbd0b2fd05e 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index c73d27de8a45..6275ebb8259e 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 63198c198edc..706c6b66dbeb 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.149 + 0.149-tw-0.25 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index b03e8e6acc3c..132567423856 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 03d54389f987..95bae83cc8a3 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index a74a27be3f8c..1911d10efcf7 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 03c448212c15..56062155bc00 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 58398e0dc6f5..ccf9b9c9eefa 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index b8d82a0b8a7c..0a38ee035f39 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 8b83af250254..2f2c445868a4 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index ea8070438a1d..8cd92ff0172f 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 72207865f38c..d81ea563c4ec 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index d305d142df4c..79da6f2cfa01 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 201d1afe7674..30147c361b34 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 8e40fc0b237d..3dfa747efde6 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 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 6da153d2a898..55fcce7db92a 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 @@ -41,14 +41,14 @@ import org.apache.hadoop.mapred.InputFormat; import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.mapred.TextInputFormat; +import org.apache.hadoop.security.UserGroupInformation; import java.io.BufferedReader; import java.io.IOException; -import java.security.PrivilegedExceptionAction; import java.io.InputStreamReader; import java.nio.charset.StandardCharsets; +import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.Deque; import java.util.List; diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index fc710346266a..162187eb1a44 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 72979a0fbfec..18d6ea61e4f9 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 0bea79df0b75..7018bfaad555 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index d962fddabf80..0f50c038f3ae 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 2be41fe12a48..0272a3419ecf 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 1f28da25ceda..a80cc82ee392 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 7387f060788c..fccde09ef8e3 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index d6e339e06e67..d7043fd17066 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 7eee85282846..a2135acd8b46 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 3c457be6f418..6914d4063014 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index b1095676e8d1..0eb5d49a788e 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 9f66fc0766f0..b39f43defae3 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.149 + 0.149-tw-0.25 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 7b84574e3ef9..68a69009a4b6 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index fb3c6827e991..f5be0b5b557c 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index fa0916fb9bf1..c0acace95105 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index f831831e0a5d..b91a02b4d835 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 4ca0a9cf7397..69119bfea46c 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 8cec5d288c1b..83f72ae8a5a3 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 2a24fdc53a19..f4c18070879f 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 355e72d34716..e22cd4d33df3 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index cae36d3a457c..8494dc7183d0 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.149 + 0.149-tw-0.25 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index d8c0c539d4e6..5cae2b5c833a 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index dbfa75787fc6..3f7b41d22be8 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149 + 0.149-tw-0.25 presto-verifier From 53dcebe3014bc91f3db8a8c009e55b189f43dcd7 Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Tue, 8 Nov 2016 15:08:12 -0800 Subject: [PATCH 094/331] avoid java version check for twitter --- .../com/facebook/presto/server/PrestoSystemRequirements.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-main/src/main/java/com/facebook/presto/server/PrestoSystemRequirements.java b/presto-main/src/main/java/com/facebook/presto/server/PrestoSystemRequirements.java index 5bbef6bc4f09..a9066b4cad2e 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/PrestoSystemRequirements.java +++ b/presto-main/src/main/java/com/facebook/presto/server/PrestoSystemRequirements.java @@ -45,7 +45,7 @@ public static void verifyJvmRequirements() failRequirement("Presto requires an Oracle or OpenJDK JVM (found %s)", vendor); } - verifyJavaVersion(); + //verifyJavaVersion(); String dataModel = System.getProperty("sun.arch.data.model"); if (!"64".equals(dataModel)) { From 488d786f3a6d9e6638113806d3ea4174bab19fdb Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 25 Oct 2016 16:52:05 -0700 Subject: [PATCH 095/331] Added shuffle for metastores in order to balance connection load --- .../presto/twitter/hive/ZookeeperServersetHiveCluster.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java index 83642e41c50f..1885f515848d 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java @@ -23,6 +23,7 @@ import javax.inject.Inject; +import java.util.Collections; import java.util.List; import static com.facebook.presto.hive.HiveErrorCode.HIVE_METASTORE_ERROR; @@ -51,6 +52,7 @@ public ZookeeperServersetHiveCluster(ZookeeperServersetMetastoreConfig config, H public HiveMetastoreClient createMetastoreClient() { List metastores = zkMetastoreMonitor.getServers(); + Collections.shuffle(metastores); TTransportException lastException = null; for (HostAndPort metastore : metastores) { try { From b33652e517b10e19567b89e0893e64e0c586760c Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 27 Oct 2016 17:49:54 -0700 Subject: [PATCH 096/331] Added log for hive connection and log for RetryDriver --- .../src/main/java/com/facebook/presto/hive/RetryDriver.java | 4 +++- .../presto/twitter/hive/ZookeeperServersetHiveCluster.java | 3 ++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/RetryDriver.java b/presto-hive/src/main/java/com/facebook/presto/hive/RetryDriver.java index d969a522a750..c2c1fac24ffc 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/RetryDriver.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/RetryDriver.java @@ -136,16 +136,18 @@ public V run(String callableName, Callable callable) return callable.call(); } catch (Exception e) { + log.debug("Failed on executing %s with attempt %d, Exception: %s", callableName, attempt, e.getMessage()); e = exceptionMapper.apply(e); for (Class clazz : exceptionWhiteList) { if (clazz.isInstance(e)) { + log.debug("Exception is in whitelist."); throw e; } } if (attempt >= maxAttempts || Duration.nanosSince(startTime).compareTo(maxRetryTime) >= 0) { + log.debug("Maximum attempts or maximum retry time reached. attempt: %d, maxAttempts: %d, duration: [%s] maxRetryTime: [%s]", attempt, maxAttempts, Duration.nanosSince(startTime).toString(), maxRetryTime.toString()); throw e; } - log.debug("Failed on executing %s with attempt %d, will retry. Exception: %s", callableName, attempt, e.getMessage()); int delayInMs = (int) Math.min(minSleepTime.toMillis() * Math.pow(scaleFactor, attempt - 1), maxSleepTime.toMillis()); TimeUnit.MILLISECONDS.sleep(delayInMs); diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java index 1885f515848d..1f1b6cad01a4 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java @@ -60,10 +60,11 @@ public HiveMetastoreClient createMetastoreClient() return clientFactory.create(metastore.getHostText(), metastore.getPort()); } catch (TTransportException e) { + log.debug("Failed connecting to Hive metastore at: " + metastore.toString()); lastException = e; } } - throw new PrestoException(HIVE_METASTORE_ERROR, "Failed connecting to Hive metastore", lastException); + throw new PrestoException(HIVE_METASTORE_ERROR, "Failed connecting to Hive metastore: " + metastores, lastException); } } From c43856986a1be1b148c602ebf41b7bfd9b77795a Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Mon, 31 Oct 2016 15:45:52 -0700 Subject: [PATCH 097/331] Used RuntimeException to replace PrestoException, prevent whitelist stops RetryDriver --- .../presto/twitter/hive/ZookeeperServersetHiveCluster.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java index 1f1b6cad01a4..ca684b269cf5 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java @@ -16,7 +16,6 @@ import com.facebook.presto.hive.HiveCluster; import com.facebook.presto.hive.HiveMetastoreClientFactory; import com.facebook.presto.hive.metastore.HiveMetastoreClient; -import com.facebook.presto.spi.PrestoException; import com.google.common.net.HostAndPort; import io.airlift.log.Logger; import org.apache.thrift.transport.TTransportException; @@ -26,7 +25,6 @@ import java.util.Collections; import java.util.List; -import static com.facebook.presto.hive.HiveErrorCode.HIVE_METASTORE_ERROR; import static java.util.Objects.requireNonNull; public class ZookeeperServersetHiveCluster @@ -65,6 +63,6 @@ public HiveMetastoreClient createMetastoreClient() } } - throw new PrestoException(HIVE_METASTORE_ERROR, "Failed connecting to Hive metastore: " + metastores, lastException); + throw new RuntimeException("Failed connecting to Hive metastore: " + metastores, lastException); } } From 43eff7d14a0fa11ef19bd526100b9216131cc664 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Mon, 31 Oct 2016 18:22:10 -0700 Subject: [PATCH 098/331] Used formatter for logs --- .../presto/twitter/hive/ZookeeperServersetHiveCluster.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java index ca684b269cf5..379352d4100e 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java @@ -54,15 +54,15 @@ public HiveMetastoreClient createMetastoreClient() TTransportException lastException = null; for (HostAndPort metastore : metastores) { try { - log.info("Connecting to metastore at: " + metastore.toString()); + log.info("Connecting to metastore at: %s", metastore.toString()); return clientFactory.create(metastore.getHostText(), metastore.getPort()); } catch (TTransportException e) { - log.debug("Failed connecting to Hive metastore at: " + metastore.toString()); + log.debug("Failed connecting to Hive metastore at: %s", metastore.toString()); lastException = e; } } - throw new RuntimeException("Failed connecting to Hive metastore: " + metastores, lastException); + throw new RuntimeException("Failed connecting to Hive metastore.", lastException); } } From abe1d34774ac0d9e387e9ab24437d332a3b06031 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 18 Nov 2016 14:15:40 -0800 Subject: [PATCH 099/331] Fix bug for tableau webapp --- .../src/main/resources/webapp/tableau/presto-connector.html | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/presto-main/src/main/resources/webapp/tableau/presto-connector.html b/presto-main/src/main/resources/webapp/tableau/presto-connector.html index 3b915fdfb4b2..f4a48c6466f9 100644 --- a/presto-main/src/main/resources/webapp/tableau/presto-connector.html +++ b/presto-main/src/main/resources/webapp/tableau/presto-connector.html @@ -2,8 +2,8 @@ - - + + + diff --git a/presto-main/src/main/resources/webapp/vendor/tableau/tableauwdc-1.1.0.js b/presto-main/src/main/resources/webapp/vendor/tableau/tableauwdc-1.1.0.js new file mode 100644 index 000000000000..11e72f9288e0 --- /dev/null +++ b/presto-main/src/main/resources/webapp/vendor/tableau/tableauwdc-1.1.0.js @@ -0,0 +1,202 @@ +//The MIT License (MIT) +// +//Copyright (c) 2015 Tableau +// +//Permission is hereby granted, free of charge, to any person obtaining a copy +//of this software and associated documentation files (the "Software"), to deal +//in the Software without restriction, including without limitation the rights +//to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +//copies of the Software, and to permit persons to whom the Software is +//furnished to do so, subject to the following conditions: +// +//The above copyright notice and this permission notice shall be included in all +//copies or substantial portions of the Software. +// +//THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +//IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +//FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +//AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +//LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +//OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +//SOFTWARE. +(function() { + + var versionNumber = "1.1.0"; + var _sourceWindow; + + if (typeof tableauVersionBootstrap === 'undefined') { + // tableau version bootstrap isn't defined. We are likely running in the simulator so init up our tableau object + tableau = { + connectionName: "", + connectionData: "", + password: "", + username: "", + incrementalExtractColumn: "", + + initCallback: function () { + _sendMessage("initCallback"); + }, + + shutdownCallback: function () { + _sendMessage("shutdownCallback"); + }, + + submit: function () { + _sendMessage("submit"); + }, + + log: function (msg) { + _sendMessage("log", {"logMsg": msg}); + }, + + headersCallback: function (fieldNames, types) { + _sendMessage("headersCallback", {"fieldNames": fieldNames, "types":types}); + }, + + dataCallback: function (data, lastRecordToken, moreData) { + _sendMessage("dataCallback", {"data": data, "lastRecordToken": lastRecordToken, "moreData": moreData}); + }, + + abortWithError: function (errorMsg) { + _sendMessage("abortWithError", {"errorMsg": errorMsg}); + } + }; + } else { // Tableau version bootstrap is defined. Let's use it + tableauVersionBootstrap.ReportVersionNumber(versionNumber); + } + + // Check if something weird happened during bootstraping. If so, just define a tableau object to we don't + // throw errors all over the place because tableau isn't defined + if (typeof tableau === "undefined") { + tableau = {} + } + + tableau.versionNumber = versionNumber; + + tableau.phaseEnum = { + interactivePhase: "interactive", + authPhase: "auth", + gatherDataPhase: "gatherData" + }; + + if (!tableau.phase) { + tableau.phase = tableau.phaseEnum.interactivePhase; + } + + // Assign the functions we always want to have available on the tableau object + tableau.makeConnector = function() { + var defaultImpls = { + init: function() { tableau.initCallback(); }, + shutdown: function() { tableau.shutdownCallback(); } + }; + return defaultImpls; + }; + + tableau.registerConnector = function (wdc) { + // do some error checking on the wdc + var functionNames = ["init", "shutdown", "getColumnHeaders", "getTableData"] + for (var ii = functionNames.length - 1; ii >= 0; ii--) { + if (typeof(wdc[functionNames[ii]]) !== "function") { + throw "The connector did not define the required function: " + functionNames[ii]; + } + }; + window._wdc = wdc; + }; + + function _sendMessage(msgName, msgData) { + var messagePayload = _buildMessagePayload(msgName, msgData); + + // Check first to see if we have a messageHandler defined to post the message to + if (typeof window.webkit != 'undefined' && + typeof window.webkit.messageHandlers != 'undefined' && + typeof window.webkit.messageHandlers.wdcHandler != 'undefined') { + + window.webkit.messageHandlers.wdcHandler.postMessage(messagePayload); + } else if (!_sourceWindow) { + throw "Looks like the WDC is calling a tableau function before tableau.init() has been called." + } else { + _sourceWindow.postMessage(messagePayload, "*"); + } + } + + function _buildMessagePayload(msgName, msgData) { + var msgObj = {"msgName": msgName, + "props": _packagePropertyValues(), + "msgData": msgData}; + return JSON.stringify(msgObj); + } + + function _packagePropertyValues() { + var propValues = {"connectionName": tableau.connectionName, + "connectionData": tableau.connectionData, + "password": tableau.password, + "username": tableau.username, + "incrementalExtractColumn": tableau.incrementalExtractColumn, + "versionNumber": tableau.versionNumber}; + return propValues; + } + + function _applyPropertyValues(props) { + if (props) { + tableau.connectionName = props.connectionName; + tableau.connectionData = props.connectionData; + tableau.password = props.password; + tableau.username = props.username; + tableau.incrementalExtractColumn = props.incrementalExtractColumn; + } + } + + function _receiveMessage(evnt) { + var wdc = window._wdc; + if (!wdc) { + throw "No WDC registered. Did you forget to call tableau.registerConnector?"; + } + if (!_sourceWindow) { + _sourceWindow = evnt.source + } + var payloadObj = JSON.parse(evnt.data); + var msgData = payloadObj.msgData; + _applyPropertyValues(payloadObj.props); + + switch(payloadObj.msgName) { + case "init": + tableau.phase = msgData.phase; + wdc.init(); + break; + case "shutdown": + wdc.shutdown(); + break; + case "getColumnHeaders": + wdc.getColumnHeaders(); + break; + case "getTableData": + wdc.getTableData(msgData.lastRecordToken); + break; + } + }; + + // Add global error handler. If there is a javascript error, this will report it to Tableau + // so that it can be reported to the user. + window.onerror = function (message, file, line, column, errorObj) { + if (tableau._hasAlreadyThrownErrorSoDontThrowAgain) { + return true; + } + var msg = message; + if(errorObj) { + msg += " stack:" + errorObj.stack; + } else { + msg += " file: " + file; + msg += " line: " + line; + } + + if (tableau && tableau.abortWithError) { + tableau.abortWithError(msg); + } else { + throw msg; + } + tableau._hasAlreadyThrownErrorSoDontThrowAgain = true; + return true; + } + + window.addEventListener('message', _receiveMessage, false); +})(); \ No newline at end of file From 35f71d297105d707c608160ac3753dae9126133e Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Tue, 22 Nov 2016 14:33:52 -0800 Subject: [PATCH 102/331] Undo erroneous merge of commit 36175ec6d47d24ff3d7519f68d3e5c0d0af0fc98 which removed the Twitter module that scribes QueryCompletionEvent --- pom.xml | 68 +++++++ presto-main/pom.xml | 18 ++ .../operator/.HttpPageBufferClient.java.swp | Bin 0 -> 16384 bytes .../facebook/presto/server/PrestoServer.java | 6 + .../presto/twitter/TwitterModuleLoader.java | 64 +++++++ .../presto/twitter/logging/QueryLogger.java | 78 ++++++++ .../presto/twitter/logging/QueryScriber.java | 169 ++++++++++++++++++ 7 files changed, 403 insertions(+) create mode 100644 presto-main/src/main/java/com/facebook/presto/operator/.HttpPageBufferClient.java.swp create mode 100644 presto-main/src/main/java/com/facebook/presto/twitter/TwitterModuleLoader.java create mode 100644 presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java create mode 100644 presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java diff --git a/pom.xml b/pom.xml index 4b36010586f6..574f605cb947 100644 --- a/pom.xml +++ b/pom.xml @@ -761,6 +761,74 @@ 2.1.7 + + + com.twitter + presto-thrift-java + 0.0.1 + + + com.twitter + util-core_2.11 + + + com.twitter + util-core-java + + + com.twitter + util-function_2.10 + + + com.twitter + util-function-java + + + commons-logging + commons-logging + + + org.scala-lang.modules + scala-parser-combinators_2.11 + + + com.twitter + scrooge-core + + + org.scala-lang + scala-library + + + org.scala-lang + scala-reflect + + + + + + com.twitter + util-logging_2.10 + 6.34.0 + + + commons-logging + commons-logging + + + + + org.scala-lang + scala-library + 2.10.6 + + + commons-logging + commons-logging + + + + diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 94b5a1b08642..d244d702d5a5 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -252,6 +252,24 @@ jgrapht-core + + + com.twitter + presto-thrift-java + + + com.twitter + util-logging_2.10 + + + org.apache.thrift + libthrift + + + org.scala-lang + scala-library + + org.testng diff --git a/presto-main/src/main/java/com/facebook/presto/operator/.HttpPageBufferClient.java.swp b/presto-main/src/main/java/com/facebook/presto/operator/.HttpPageBufferClient.java.swp new file mode 100644 index 0000000000000000000000000000000000000000..4527e97497ec4ac48a06fca9116b4c57bdad80d0 GIT binary patch literal 16384 zcmeHN%Wot{8E;quAtVG5ffNZuVJ_Za*DML)R)E{KE!2oQ&`Z~`PwhyxN*A|Y`|{s2I5Ah20})sNYE?RZu!7q(jZ zP4`rN_0?CuuO44@SNGBtXODcEK4al}pJjdb_WHHQKk)p8KYh!x)R7#pG}So^FEv~r zbmR4N#lKcc)P^!SPGjctTAc7y$?~>7U?QreiC@bfZm<(p^W~t{gJf6A<4Rh_JYh;E zwH>A6eb(ojS+B>FMksiss4lk9T$5=Sct9C=%zEbOPh8TC|Jce$$rJBCctCkD%{B}e z1`Gp+0mFb{z%XDKFbq7H45;{F>v=T&?Pc@#%KO`GHFkl!k3>XIfFBxEgWnDqbUeX(7eE+ZZ|F<5utX}{pz%H-@ zd=Pj7_}zOg>l?rZ@b||o>kq*1fgb=b178O^z>~mV9<{9B0xtp)@GNivcnurz?*Iv~ z1$-Fz$0Lvhz6nU+^T3CIH{WepuL3uLp93!fN5E%+&j3#Xw;#5w*MVOFKLoxF41sIF zCxN%#1s%Xkzz{eAE&+dgr)7N)_#*Hq@auhA90p5yM-y~fV+y3$zao)iTv3G5`1Nkayl&eOS!Wec@! z9&B%;t&L{ewGX%JPSf66MrU;>D0lhHp&*?w8XDA3IJ)sf+W)4CyuaLYJ>_`b&wE1E zd-prG>$MC2IW5<#w;J}LvlKOp^zGL#qf@kZ?bb33oU5s2U&8>qAL{n%*A89h3v);6 z^k7+JR@I%_d;3k>TS{Fwl>RZz-RANH{T$sT$p0?GQQ@{d9th^0#TdM$=QPu9Ir&tx zus*9+40T^dewHNq02_|B4TMkY`K>9V{;1aT#hpm;{@4sNw}dAK{2&r{rs26#u7AXX zEaZWG%KZ$-?ZCx}@bAq2b4zV!3NzLmi`%ghGGgIf8dPPY0dopBMLLzNC-;MnGbxLO z(TUhs*1nY>0QQ_i~KR0E*5IAX|yO6j-Ite=$NHE1*@uj&s(hwbhy81UKdY+a-X3P(Z#Y5SrP1zssv|~n($2V0_;z`+L zNuSTU@tlHXNHovrdG(wxm&Z5fiRlpJAQ5F0w2X8SvMnRoC?Y>yMa9n40K5w&x$`C4&q!D4xs?Yr9cc@^n(XT!2^ zTKX7IWS&?R`fOO_sTx2+F%ZRcl#(n$@JrMYCryl6Yz(NDa5*1OBRyHjq%rqJ4||M| zMg2@y*q0|fiO@WP$aNsn6q83v5D98ER~>Jsec+L+^-ibW@*LYG?G8bpz2$gLyM=$- zq~5wtcAeJN8sP$FB!r*p7&j%T5}Oh9;2<|Cz#N~5Zl-uJYcvrl85fWdOducHdBv8~nBP>Q8zIHzvHPp4Te!`KbBoPrACPJZgYb2FDHN^e| zYP2SO=~>}(wc2nvqzn!?;uO4w9wX8)#xrz)GGkPuey(8@98^=VS=(#>l^z}EtDf6g`Ff39 z!OjOBBe0RI>Rzx?7FxNuVbz|p^!{I9AOBN8@Bgt4&#ymYpZ`bTN5Bh!3w#8419HCs zZUR38bh)1Z<}wTz1`Gp+0mFb{z%XDKFbo(53 vX7s$MfGUOa`=nsP@uEG#ON{X^i`e0d^p4z@NA|~A7q2ZyAv-FxdeQn1&2=?# literal 0 HcmV?d00001 diff --git a/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java b/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java index 3aa94c7f9317..7e1bbcd031c3 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java +++ b/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java @@ -103,6 +103,7 @@ public void run() new ServerMainModule(sqlParserOptions), new GracefulShutdownModule()); + modules.addAll(getAdditionalModules()); Bootstrap app = new Bootstrap(modules.build()); try { @@ -131,6 +132,11 @@ public void run() } } + protected Iterable getAdditionalModules() + { + return com.facebook.presto.twitter.TwitterModuleLoader.getAdditionalModules(); + } + private static void updateDatasources(Announcer announcer, Metadata metadata, ServerConfig serverConfig, NodeSchedulerConfig schedulerConfig) { // get existing announcement diff --git a/presto-main/src/main/java/com/facebook/presto/twitter/TwitterModuleLoader.java b/presto-main/src/main/java/com/facebook/presto/twitter/TwitterModuleLoader.java new file mode 100644 index 000000000000..eb23a7650a09 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/twitter/TwitterModuleLoader.java @@ -0,0 +1,64 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.twitter; + +import com.facebook.presto.event.EventProcessor; +import com.facebook.presto.event.query.QueryCompletionEvent; +import com.facebook.presto.event.query.QueryCreatedEvent; +import com.facebook.presto.event.query.QueryEventHandler; +import com.facebook.presto.event.query.SplitCompletionEvent; +import com.facebook.presto.twitter.logging.QueryLogger; +import com.facebook.presto.twitter.logging.QueryScriber; +import com.google.common.collect.ImmutableList; +import com.google.inject.Module; +import com.google.inject.Scopes; +import com.google.inject.TypeLiteral; +import com.google.inject.multibindings.Multibinder; +import io.airlift.event.client.EventClient; + +/** + * Loader that initializes custom Twitter code to inject into Presto. Whenever + * possible we should use this pattern to inject custom functionality, since it + * makes it easier to differentiate our patches from the core OS code. + * + * If the functionality we wish to add/override isn't currently possible to via + * overriding a guice module, we should contribute the necessary modules/interfaces + * into the OS Presto code base to make it possible. + */ +public class TwitterModuleLoader +{ + private TwitterModuleLoader() + { + } + + public static Iterable getAdditionalModules() + { + return ImmutableList.of( + binder -> Multibinder.newSetBinder(binder, EventClient.class) + .addBinding() + .to(EventProcessor.class) + .in(Scopes.SINGLETON), + binder -> Multibinder.newSetBinder(binder, new TypeLiteral>() {}), + binder -> Multibinder.newSetBinder(binder, new TypeLiteral>() {}), + binder -> Multibinder.newSetBinder(binder, new TypeLiteral>(){}) + .addBinding() + .to(new TypeLiteral(){}) + .in(Scopes.SINGLETON), + binder -> Multibinder.newSetBinder(binder, new TypeLiteral>(){}) + .addBinding() + .to(QueryScriber.class) + .in(Scopes.SINGLETON) + ); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java new file mode 100644 index 000000000000..02edd8da2440 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java @@ -0,0 +1,78 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.twitter.logging; + +import com.facebook.presto.event.query.QueryCompletionEvent; +import com.facebook.presto.event.query.QueryEventHandler; +import com.facebook.presto.spi.StandardErrorCode; +import io.airlift.log.Logger; +import io.airlift.units.Duration; + +import java.util.concurrent.TimeUnit; + +/** + * Class that logs query events to a file + */ +public class QueryLogger implements QueryEventHandler +{ + private static final int MAX_QUERY_LENGTH = 1000; + private static final String DASH = "-"; + private static final String COLON = ":"; + private static final String SPACE = " "; + private static final String ELIPSIS = "..."; + private static final String QUERY_COMPLETION = "QueryCompletion"; + + private static final Logger log = Logger.get(QueryLogger.class); + + @Override + public void handle(QueryCompletionEvent event) + { + String errorType = DASH; + String errorCode = DASH; + if (event.getErrorCode() != null) { + errorType = StandardErrorCode.toErrorType(event.getErrorCode()).toString(); + if (event.getErrorCodeName() != null) { + errorCode = event.getErrorCodeName() + COLON + event.getErrorCode(); + } + } + + Duration duration = (new Duration( + event.getQueryWallTimeMs(), TimeUnit.MILLISECONDS)) + .convertToMostSuccinctTimeUnit(); + + log.info(String.format("%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s", + QUERY_COMPLETION, event.getQueryId(), toLogValue(event.getRemoteClientAddress()), + event.getQueryState(), errorType, errorCode, event.getUser(), duration, + event.getSplits(), event.getTotalRows(), event.getTotalBytes(), + cleanseAndTrimQuery(event.getQuery()))); + } + + private static String toLogValue(Object object) + { + if (object == null) { + return DASH; + } + else { + return object.toString(); + } + } + + private static String cleanseAndTrimQuery(String query) + { + if (query.length() > MAX_QUERY_LENGTH) { + query = query.substring(0, MAX_QUERY_LENGTH) + ELIPSIS; + } + return query.replace(System.getProperty("line.separator"), SPACE); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java new file mode 100644 index 000000000000..5e55aa1ac195 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java @@ -0,0 +1,169 @@ +/* + * 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.twitter.logging; + +import com.facebook.presto.event.query.QueryCompletionEvent; +import com.facebook.presto.event.query.QueryEventHandler; +import com.google.common.base.Optional; +import com.twitter.logging.BareFormatter$; +import com.twitter.logging.Level; +import com.twitter.logging.QueueingHandler; +import com.twitter.logging.ScribeHandler; +import com.twitter.presto.thriftjava.QueryState; +import io.airlift.log.Logger; +import org.apache.thrift.TBase; +import org.apache.thrift.TException; +import org.apache.thrift.TSerializer; + +import java.util.Base64; +import java.util.logging.LogRecord; + +/** + * Class that scribes query completion events + */ +public class QueryScriber implements QueryEventHandler +{ + private static final String SCRIBE_CATEGORY = "presto_query_completion"; + private static final int MAX_QUEUE_SIZE = 1000; + + private static final Logger log = Logger.get(QueryScriber.class); + + private QueueingHandler queueingHandler; + + // TSerializer is not thread safe + private final ThreadLocal serializer = new ThreadLocal() + { + @Override protected TSerializer initialValue() + { + return new TSerializer(); + } + }; + + public QueryScriber() + { + ScribeHandler scribeHandler = new ScribeHandler( + ScribeHandler.DefaultHostname(), + ScribeHandler.DefaultPort(), + SCRIBE_CATEGORY, + ScribeHandler.DefaultBufferTime(), + ScribeHandler.DefaultConnectBackoff(), + ScribeHandler.DefaultMaxMessagesPerTransaction(), + ScribeHandler.DefaultMaxMessagesToBuffer(), + BareFormatter$.MODULE$, + scala.Option.apply((Level) null)); + queueingHandler = new QueueingHandler(scribeHandler, MAX_QUEUE_SIZE); + } + + @Override + public void handle(QueryCompletionEvent event) + { + com.twitter.presto.thriftjava.QueryCompletionEvent thriftEvent = toThriftQueryCompletionEvent(event); + Optional message = serializeThriftToString(thriftEvent); + + if (message.isPresent()) { + LogRecord logRecord = new LogRecord(Level.ALL, message.get()); + queueingHandler.publish(logRecord); + } + else { + log.warn("Unable to serialize QueryCompletionEvent: " + event); + } + } + + /** + * Serialize a thrift object to bytes, compress, then encode as a base64 string. + */ + private Optional serializeThriftToString(TBase thriftMessage) + { + try { + return Optional.of( + Base64.getEncoder().encodeToString(serializer.get().serialize(thriftMessage))); + } + catch (TException e) { + log.warn(e, "Could not serialize thrift object" + thriftMessage); + return Optional.absent(); + } + } + + private static com.twitter.presto.thriftjava.QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletionEvent event) + { + com.twitter.presto.thriftjava.QueryCompletionEvent thriftEvent = + new com.twitter.presto.thriftjava.QueryCompletionEvent(); + + thriftEvent.query_id = event.getQueryId(); + thriftEvent.transaction_id = event.getTransactionId(); + thriftEvent.user = event.getUser(); + thriftEvent.principal = event.getPrincipal(); + thriftEvent.source = event.getSource(); + thriftEvent.server_version = event.getServerVersion(); + thriftEvent.environment = event.getEnvironment(); + thriftEvent.catalog = event.getCatalog(); + thriftEvent.schema = event.getSchema(); + thriftEvent.remote_client_address = event.getRemoteClientAddress(); + thriftEvent.user_agent = event.getUserAgent(); + thriftEvent.query_state = QueryState.valueOf(event.getQueryState()); + thriftEvent.uri = event.getUri(); + thriftEvent.field_names = event.getFieldNames(); + thriftEvent.query = event.getQuery(); + thriftEvent.create_time_ms = event.getCreateTime().getMillis(); + thriftEvent.execution_start_time_ms = event.getExecutionStartTime().getMillis(); + thriftEvent.end_time_ms = event.getEndTime().getMillis(); + thriftEvent.queued_time_ms = event.getQueuedTimeMs(); + if (event.getAnalysisTimeMs() != null) { + thriftEvent.analysis_time_ms = event.getAnalysisTimeMs(); + } + if (event.getDistributedPlanningTimeMs() != null) { + thriftEvent.distributed_planning_time_ms = event.getDistributedPlanningTimeMs(); + } + if (event.getTotalSplitWallTimeMs() != null) { + thriftEvent.total_split_wall_time_ms = event.getTotalSplitWallTimeMs(); + } + if (event.getTotalSplitCpuTimeMs() != null) { + thriftEvent.total_split_cpu_time_ms = event.getTotalSplitCpuTimeMs(); + } + if (event.getTotalBytes() != null) { + thriftEvent.total_bytes = event.getTotalBytes(); + } + if (event.getTotalRows() != null) { + thriftEvent.total_rows = event.getTotalRows(); + } + thriftEvent.splits = event.getSplits(); + if (event.getErrorCode() != null) { + thriftEvent.error_code_id = event.getErrorCode(); + } + thriftEvent.error_code_name = event.getErrorCodeName(); + thriftEvent.failure_type = event.getFailureType(); + thriftEvent.failure_message = event.getFailureMessage(); + thriftEvent.failure_task = event.getFailureTask(); + thriftEvent.failure_host = event.getFailureHost(); + thriftEvent.output_stage_json = event.getOutputStageJson(); + thriftEvent.failures_json = event.getFailuresJson(); + thriftEvent.inputs_json = event.getInputsJson(); + thriftEvent.session_properties_json = event.getSessionPropertiesJson(); + thriftEvent.query_wall_time_ms = event.getQueryWallTimeMs(); + if (event.getBytesPerSec() != null) { + thriftEvent.bytes_per_sec = event.getBytesPerSec(); + } + if (event.getBytesPerCpuSec() != null) { + thriftEvent.bytes_per_cpu_sec = event.getBytesPerCpuSec(); + } + if (event.getRowsPerSec() != null) { + thriftEvent.rows_per_sec = event.getRowsPerSec(); + } + if (event.getRowsPerCpuSec() != null) { + thriftEvent.rows_per_cpu_sec = event.getRowsPerCpuSec(); + } + + return thriftEvent; + } +} From 30630793e4ad467a2f9aafef4dc8ec64417a12a0 Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Tue, 22 Nov 2016 14:35:59 -0800 Subject: [PATCH 103/331] update tag to 0.149-tw-0.27 --- pom.xml | 4 ++-- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 40 files changed, 41 insertions(+), 41 deletions(-) diff --git a/pom.xml b/pom.xml index 574f605cb947..c31176988c59 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.149-tw-0.26 + 0.149-tw-0.27 diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 86a265fda82a..09465ba6c003 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 2341ce4f1b56..0339a4e61b2b 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 571824609031..7e26a4b4544a 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index d676a6d61541..1bce4669b733 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.149-tw-0.26 + 0.149-tw-0.27 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 8102a09248eb..8f5e60ab8f67 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 4c5d0cd8499e..147e19a1d0be 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index f7de6cf287f6..8fca2daa6285 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index ee11160dd65b..4ce75a91f29e 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index ac42c8769498..fc675ad793c1 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 1f41107e5199..f1e97a471056 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index f824320b74da..e3a043887564 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 372d9932f9a4..56a91e857267 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index a60ed7801af2..14232f3a3b5b 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index efca94d80ce5..636a011ee5e1 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 77b4e787fabf..298456be757c 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index f32f810c7d41..f3b0ec562a81 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index ba717e41d02f..88e5f9bb334e 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 5d50049084fd..13f8dd9ec3f9 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index c9309a2fae26..b644db550652 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 02179d1d9e7c..8803cdbc9630 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index d244d702d5a5..02c1d2323f44 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 6a88d759675a..0260655c0298 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 73262f63ffaf..28d973ed89c7 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 7f90f9b8282a..5e407459011c 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 0efc7dac4577..30cc3fa661f8 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index a20ab814f118..4ba0e816b0f5 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 40ece3041bcb..de3ff795dbec 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index e337e6cc6f01..9a9e22b52ad1 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.149-tw-0.26 + 0.149-tw-0.27 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 357ae8bcc93d..95c4461e9845 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index dce3b54ffd2f..c7439b37d553 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 8d11658416aa..1ccdab8fce4c 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index f4ff91a09ca5..b471f8b603f2 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index dc787efaf891..6ceae8a36b78 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index f87acdfc90a1..e26987fdaf37 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 2ac12cb7f520..6287071b0c71 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 58a857dfbfa6..46b9e6f1bcc4 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 9b673a003d44..54849774f6c2 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.149-tw-0.26 + 0.149-tw-0.27 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 1e78b6278ed6..278763709289 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index ca874156cb9a..50e222cdc4fb 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.26 + 0.149-tw-0.27 presto-verifier From 5f21b8269ba4a9a180081f872db5c0af3a323b54 Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Tue, 22 Nov 2016 15:08:13 -0800 Subject: [PATCH 104/331] take out swap file --- .../operator/.HttpPageBufferClient.java.swp | Bin 16384 -> 0 bytes 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 presto-main/src/main/java/com/facebook/presto/operator/.HttpPageBufferClient.java.swp diff --git a/presto-main/src/main/java/com/facebook/presto/operator/.HttpPageBufferClient.java.swp b/presto-main/src/main/java/com/facebook/presto/operator/.HttpPageBufferClient.java.swp deleted file mode 100644 index 4527e97497ec4ac48a06fca9116b4c57bdad80d0..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16384 zcmeHN%Wot{8E;quAtVG5ffNZuVJ_Za*DML)R)E{KE!2oQ&`Z~`PwhyxN*A|Y`|{s2I5Ah20})sNYE?RZu!7q(jZ zP4`rN_0?CuuO44@SNGBtXODcEK4al}pJjdb_WHHQKk)p8KYh!x)R7#pG}So^FEv~r zbmR4N#lKcc)P^!SPGjctTAc7y$?~>7U?QreiC@bfZm<(p^W~t{gJf6A<4Rh_JYh;E zwH>A6eb(ojS+B>FMksiss4lk9T$5=Sct9C=%zEbOPh8TC|Jce$$rJBCctCkD%{B}e z1`Gp+0mFb{z%XDKFbq7H45;{F>v=T&?Pc@#%KO`GHFkl!k3>XIfFBxEgWnDqbUeX(7eE+ZZ|F<5utX}{pz%H-@ zd=Pj7_}zOg>l?rZ@b||o>kq*1fgb=b178O^z>~mV9<{9B0xtp)@GNivcnurz?*Iv~ z1$-Fz$0Lvhz6nU+^T3CIH{WepuL3uLp93!fN5E%+&j3#Xw;#5w*MVOFKLoxF41sIF zCxN%#1s%Xkzz{eAE&+dgr)7N)_#*Hq@auhA90p5yM-y~fV+y3$zao)iTv3G5`1Nkayl&eOS!Wec@! z9&B%;t&L{ewGX%JPSf66MrU;>D0lhHp&*?w8XDA3IJ)sf+W)4CyuaLYJ>_`b&wE1E zd-prG>$MC2IW5<#w;J}LvlKOp^zGL#qf@kZ?bb33oU5s2U&8>qAL{n%*A89h3v);6 z^k7+JR@I%_d;3k>TS{Fwl>RZz-RANH{T$sT$p0?GQQ@{d9th^0#TdM$=QPu9Ir&tx zus*9+40T^dewHNq02_|B4TMkY`K>9V{;1aT#hpm;{@4sNw}dAK{2&r{rs26#u7AXX zEaZWG%KZ$-?ZCx}@bAq2b4zV!3NzLmi`%ghGGgIf8dPPY0dopBMLLzNC-;MnGbxLO z(TUhs*1nY>0QQ_i~KR0E*5IAX|yO6j-Ite=$NHE1*@uj&s(hwbhy81UKdY+a-X3P(Z#Y5SrP1zssv|~n($2V0_;z`+L zNuSTU@tlHXNHovrdG(wxm&Z5fiRlpJAQ5F0w2X8SvMnRoC?Y>yMa9n40K5w&x$`C4&q!D4xs?Yr9cc@^n(XT!2^ zTKX7IWS&?R`fOO_sTx2+F%ZRcl#(n$@JrMYCryl6Yz(NDa5*1OBRyHjq%rqJ4||M| zMg2@y*q0|fiO@WP$aNsn6q83v5D98ER~>Jsec+L+^-ibW@*LYG?G8bpz2$gLyM=$- zq~5wtcAeJN8sP$FB!r*p7&j%T5}Oh9;2<|Cz#N~5Zl-uJYcvrl85fWdOducHdBv8~nBP>Q8zIHzvHPp4Te!`KbBoPrACPJZgYb2FDHN^e| zYP2SO=~>}(wc2nvqzn!?;uO4w9wX8)#xrz)GGkPuey(8@98^=VS=(#>l^z}EtDf6g`Ff39 z!OjOBBe0RI>Rzx?7FxNuVbz|p^!{I9AOBN8@Bgt4&#ymYpZ`bTN5Bh!3w#8419HCs zZUR38bh)1Z<}wTz1`Gp+0mFb{z%XDKFbo(53 vX7s$MfGUOa`=nsP@uEG#ON{X^i`e0d^p4z@NA|~A7q2ZyAv-FxdeQn1&2=?# From 1613f3c252fa2a26c3bfd9e357a760c41555f8cd Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Wed, 30 Nov 2016 22:34:24 -0800 Subject: [PATCH 105/331] update tag to 0.157 to avoid unnecessary conflicts --- pom.xml | 4 ++-- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 40 files changed, 41 insertions(+), 41 deletions(-) diff --git a/pom.xml b/pom.xml index c31176988c59..a83b82f86e06 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.149-tw-0.27 + 0.157 diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 09465ba6c003..aba5eb525704 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 0339a4e61b2b..f8dbdf7d1b95 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 7e26a4b4544a..6fddcde5e32d 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 1bce4669b733..ed9d3f7ad152 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.149-tw-0.27 + 0.157 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 8f5e60ab8f67..4bb0688136f1 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 147e19a1d0be..328239356ba4 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 8fca2daa6285..945ddbb8ddeb 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 4ce75a91f29e..0298db60e938 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index fc675ad793c1..04211daf674c 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index f1e97a471056..c2dd2d1c4615 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index e3a043887564..0e9b8fa8278c 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 56a91e857267..80db7ffef80e 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 14232f3a3b5b..3dcb24fedbf5 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 636a011ee5e1..ec1d70ba482b 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 298456be757c..807618da781d 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index f3b0ec562a81..2b9c0c165a6a 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 88e5f9bb334e..e9318e30acae 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 13f8dd9ec3f9..4eeaf29519af 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index b644db550652..77e40b94d371 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 8803cdbc9630..0ef69e5e32a6 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 02c1d2323f44..cf88c221ca21 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 0260655c0298..db2c2ce25228 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 28d973ed89c7..365fcc80b528 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 5e407459011c..26d655a500af 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 30cc3fa661f8..af785987e76f 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 4ba0e816b0f5..6d68e76e7beb 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-parser diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index de3ff795dbec..b534f0672eb3 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 9a9e22b52ad1..be29bb33055c 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.149-tw-0.27 + 0.157 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 95c4461e9845..f7f7825a51af 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-raptor diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index c7439b37d553..80619687049e 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 1ccdab8fce4c..df7bba38b1d6 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-redis diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index b471f8b603f2..ad773d1d327a 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 6ceae8a36b78..bcea4da80b02 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index e26987fdaf37..c9913e9ea95a 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 6287071b0c71..38533a5d3c8e 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 46b9e6f1bcc4..b664281984a8 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 54849774f6c2..c6bbeb2d5d08 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.149-tw-0.27 + 0.157 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 278763709289..10c39761f7b4 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 50e222cdc4fb..8c2cf5adabdc 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157 presto-verifier From 50b4884e4944bff8e18db773c1f5e7ab9bb9018d Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Thu, 1 Dec 2016 13:49:43 -0800 Subject: [PATCH 106/331] update tag --- pom.xml | 5 +++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 46 files changed, 49 insertions(+), 48 deletions(-) diff --git a/pom.xml b/pom.xml index 0701f6ac3b43..77381e4f9179 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.157 + 0.157-tw-0.28 @@ -91,6 +91,7 @@ presto-hive-cdh5 presto-teradata-functions presto-example-http + twitter-eventlistener-plugin presto-local-file presto-tpch presto-raptor diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 3b26ff81741e..c626c3ad70b2 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 8ccf2972c75f..a500d24dd11f 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 75242cfb9a05..8761d3638bc6 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 1d806e5922c5..73936549aa05 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 6fddcde5e32d..4607b9c36eb8 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 38f595629841..d7b20168f9d9 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157 + 0.157-tw-0.28 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index c24904ad5f70..8ccb6e9fab7a 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 328239356ba4..e5c8a761868a 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 9ae5ae5a199b..f49684f0083f 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 0298db60e938..e2e5a57835bf 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 04211daf674c..4f53264a9894 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 15d4926794b6..0cffa89f962b 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index d1d735d3236a..533ffeda6853 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 3c60601e9c1c..2af0998f777e 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index cf758f778024..03c62da1a4db 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 56c448a6db72..525324657f8d 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index af0312d7c184..7dfdfd71281c 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 9ee04fa53548..94c8f9f305e7 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index b49325fc1412..059491bbb6f6 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 927a48fafca0..68f97938c645 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 242720afd38b..e1d4ffd5735a 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 319fcc74d9b2..9c387bfc64c5 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index f37725e8a11c..93da37fd6fc6 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 69e9cf92500a..008c82fdbe20 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index f499be4b25cb..7087a7cdab75 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index dbda79095e31..874270eb3e39 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index af785987e76f..40c93e82bd6e 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 6d68e76e7beb..a73eefd181a2 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 47396fe8eaf2..e3e52ccc9c94 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 4abebd8ebff1..7d6c08ae0f32 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 086ea98b80d0..699d996b9cf9 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157 + 0.157-tw-0.28 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 48885f10e414..fc953510fc6e 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index dcaa2b1a50f5..b550cf7cfe49 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 499cb961d1ac..caa9be15c786 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index c788b17af12c..33eabb69a5fa 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index cff6840b7dca..0dec7412e1e5 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 0417661668dd..dedf4daf4cb8 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index bcea4da80b02..51b1da095a93 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 8e4f18a6e8eb..8bb450c60706 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 469d5c15dda9..2b9ec4feb53f 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index b664281984a8..8ef0f28df850 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 594e2dc3ef03..0923fa884822 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157 + 0.157-tw-0.28 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 81fe59b71edc..9482bda29427 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index b17d6060f7f1..539a9fc3f03f 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157 + 0.157-tw-0.28 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 2dfb997a2475..a38d41663d1f 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.154-tw-0.24 + 0.157-tw-0.28 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.154-tw-0.24 + 0.157-tw-0.28 provided From 56fc85793630d262cddc97986bdb6dc410112b51 Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Thu, 1 Dec 2016 13:59:49 -0800 Subject: [PATCH 107/331] remove current impl of twitter event scriber - upgrade to 0.157 part 1 of 3 --- pom.xml | 68 ------- presto-main/pom.xml | 18 -- .../facebook/presto/server/PrestoServer.java | 6 - .../presto/twitter/TwitterModuleLoader.java | 64 ------- .../presto/twitter/logging/QueryLogger.java | 78 -------- .../presto/twitter/logging/QueryScriber.java | 169 ------------------ 6 files changed, 403 deletions(-) delete mode 100644 presto-main/src/main/java/com/facebook/presto/twitter/TwitterModuleLoader.java delete mode 100644 presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java delete mode 100644 presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java diff --git a/pom.xml b/pom.xml index c31176988c59..8057e9ac0c7f 100644 --- a/pom.xml +++ b/pom.xml @@ -761,74 +761,6 @@ 2.1.7 - - - com.twitter - presto-thrift-java - 0.0.1 - - - com.twitter - util-core_2.11 - - - com.twitter - util-core-java - - - com.twitter - util-function_2.10 - - - com.twitter - util-function-java - - - commons-logging - commons-logging - - - org.scala-lang.modules - scala-parser-combinators_2.11 - - - com.twitter - scrooge-core - - - org.scala-lang - scala-library - - - org.scala-lang - scala-reflect - - - - - - com.twitter - util-logging_2.10 - 6.34.0 - - - commons-logging - commons-logging - - - - - org.scala-lang - scala-library - 2.10.6 - - - commons-logging - commons-logging - - - - diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 02c1d2323f44..7fa35421c891 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -252,24 +252,6 @@ jgrapht-core - - - com.twitter - presto-thrift-java - - - com.twitter - util-logging_2.10 - - - org.apache.thrift - libthrift - - - org.scala-lang - scala-library - - org.testng diff --git a/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java b/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java index 7e1bbcd031c3..3aa94c7f9317 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java +++ b/presto-main/src/main/java/com/facebook/presto/server/PrestoServer.java @@ -103,7 +103,6 @@ public void run() new ServerMainModule(sqlParserOptions), new GracefulShutdownModule()); - modules.addAll(getAdditionalModules()); Bootstrap app = new Bootstrap(modules.build()); try { @@ -132,11 +131,6 @@ public void run() } } - protected Iterable getAdditionalModules() - { - return com.facebook.presto.twitter.TwitterModuleLoader.getAdditionalModules(); - } - private static void updateDatasources(Announcer announcer, Metadata metadata, ServerConfig serverConfig, NodeSchedulerConfig schedulerConfig) { // get existing announcement diff --git a/presto-main/src/main/java/com/facebook/presto/twitter/TwitterModuleLoader.java b/presto-main/src/main/java/com/facebook/presto/twitter/TwitterModuleLoader.java deleted file mode 100644 index eb23a7650a09..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/twitter/TwitterModuleLoader.java +++ /dev/null @@ -1,64 +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.twitter; - -import com.facebook.presto.event.EventProcessor; -import com.facebook.presto.event.query.QueryCompletionEvent; -import com.facebook.presto.event.query.QueryCreatedEvent; -import com.facebook.presto.event.query.QueryEventHandler; -import com.facebook.presto.event.query.SplitCompletionEvent; -import com.facebook.presto.twitter.logging.QueryLogger; -import com.facebook.presto.twitter.logging.QueryScriber; -import com.google.common.collect.ImmutableList; -import com.google.inject.Module; -import com.google.inject.Scopes; -import com.google.inject.TypeLiteral; -import com.google.inject.multibindings.Multibinder; -import io.airlift.event.client.EventClient; - -/** - * Loader that initializes custom Twitter code to inject into Presto. Whenever - * possible we should use this pattern to inject custom functionality, since it - * makes it easier to differentiate our patches from the core OS code. - * - * If the functionality we wish to add/override isn't currently possible to via - * overriding a guice module, we should contribute the necessary modules/interfaces - * into the OS Presto code base to make it possible. - */ -public class TwitterModuleLoader -{ - private TwitterModuleLoader() - { - } - - public static Iterable getAdditionalModules() - { - return ImmutableList.of( - binder -> Multibinder.newSetBinder(binder, EventClient.class) - .addBinding() - .to(EventProcessor.class) - .in(Scopes.SINGLETON), - binder -> Multibinder.newSetBinder(binder, new TypeLiteral>() {}), - binder -> Multibinder.newSetBinder(binder, new TypeLiteral>() {}), - binder -> Multibinder.newSetBinder(binder, new TypeLiteral>(){}) - .addBinding() - .to(new TypeLiteral(){}) - .in(Scopes.SINGLETON), - binder -> Multibinder.newSetBinder(binder, new TypeLiteral>(){}) - .addBinding() - .to(QueryScriber.class) - .in(Scopes.SINGLETON) - ); - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java deleted file mode 100644 index 02edd8da2440..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryLogger.java +++ /dev/null @@ -1,78 +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.twitter.logging; - -import com.facebook.presto.event.query.QueryCompletionEvent; -import com.facebook.presto.event.query.QueryEventHandler; -import com.facebook.presto.spi.StandardErrorCode; -import io.airlift.log.Logger; -import io.airlift.units.Duration; - -import java.util.concurrent.TimeUnit; - -/** - * Class that logs query events to a file - */ -public class QueryLogger implements QueryEventHandler -{ - private static final int MAX_QUERY_LENGTH = 1000; - private static final String DASH = "-"; - private static final String COLON = ":"; - private static final String SPACE = " "; - private static final String ELIPSIS = "..."; - private static final String QUERY_COMPLETION = "QueryCompletion"; - - private static final Logger log = Logger.get(QueryLogger.class); - - @Override - public void handle(QueryCompletionEvent event) - { - String errorType = DASH; - String errorCode = DASH; - if (event.getErrorCode() != null) { - errorType = StandardErrorCode.toErrorType(event.getErrorCode()).toString(); - if (event.getErrorCodeName() != null) { - errorCode = event.getErrorCodeName() + COLON + event.getErrorCode(); - } - } - - Duration duration = (new Duration( - event.getQueryWallTimeMs(), TimeUnit.MILLISECONDS)) - .convertToMostSuccinctTimeUnit(); - - log.info(String.format("%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s\t%s", - QUERY_COMPLETION, event.getQueryId(), toLogValue(event.getRemoteClientAddress()), - event.getQueryState(), errorType, errorCode, event.getUser(), duration, - event.getSplits(), event.getTotalRows(), event.getTotalBytes(), - cleanseAndTrimQuery(event.getQuery()))); - } - - private static String toLogValue(Object object) - { - if (object == null) { - return DASH; - } - else { - return object.toString(); - } - } - - private static String cleanseAndTrimQuery(String query) - { - if (query.length() > MAX_QUERY_LENGTH) { - query = query.substring(0, MAX_QUERY_LENGTH) + ELIPSIS; - } - return query.replace(System.getProperty("line.separator"), SPACE); - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java b/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java deleted file mode 100644 index 5e55aa1ac195..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/twitter/logging/QueryScriber.java +++ /dev/null @@ -1,169 +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.twitter.logging; - -import com.facebook.presto.event.query.QueryCompletionEvent; -import com.facebook.presto.event.query.QueryEventHandler; -import com.google.common.base.Optional; -import com.twitter.logging.BareFormatter$; -import com.twitter.logging.Level; -import com.twitter.logging.QueueingHandler; -import com.twitter.logging.ScribeHandler; -import com.twitter.presto.thriftjava.QueryState; -import io.airlift.log.Logger; -import org.apache.thrift.TBase; -import org.apache.thrift.TException; -import org.apache.thrift.TSerializer; - -import java.util.Base64; -import java.util.logging.LogRecord; - -/** - * Class that scribes query completion events - */ -public class QueryScriber implements QueryEventHandler -{ - private static final String SCRIBE_CATEGORY = "presto_query_completion"; - private static final int MAX_QUEUE_SIZE = 1000; - - private static final Logger log = Logger.get(QueryScriber.class); - - private QueueingHandler queueingHandler; - - // TSerializer is not thread safe - private final ThreadLocal serializer = new ThreadLocal() - { - @Override protected TSerializer initialValue() - { - return new TSerializer(); - } - }; - - public QueryScriber() - { - ScribeHandler scribeHandler = new ScribeHandler( - ScribeHandler.DefaultHostname(), - ScribeHandler.DefaultPort(), - SCRIBE_CATEGORY, - ScribeHandler.DefaultBufferTime(), - ScribeHandler.DefaultConnectBackoff(), - ScribeHandler.DefaultMaxMessagesPerTransaction(), - ScribeHandler.DefaultMaxMessagesToBuffer(), - BareFormatter$.MODULE$, - scala.Option.apply((Level) null)); - queueingHandler = new QueueingHandler(scribeHandler, MAX_QUEUE_SIZE); - } - - @Override - public void handle(QueryCompletionEvent event) - { - com.twitter.presto.thriftjava.QueryCompletionEvent thriftEvent = toThriftQueryCompletionEvent(event); - Optional message = serializeThriftToString(thriftEvent); - - if (message.isPresent()) { - LogRecord logRecord = new LogRecord(Level.ALL, message.get()); - queueingHandler.publish(logRecord); - } - else { - log.warn("Unable to serialize QueryCompletionEvent: " + event); - } - } - - /** - * Serialize a thrift object to bytes, compress, then encode as a base64 string. - */ - private Optional serializeThriftToString(TBase thriftMessage) - { - try { - return Optional.of( - Base64.getEncoder().encodeToString(serializer.get().serialize(thriftMessage))); - } - catch (TException e) { - log.warn(e, "Could not serialize thrift object" + thriftMessage); - return Optional.absent(); - } - } - - private static com.twitter.presto.thriftjava.QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletionEvent event) - { - com.twitter.presto.thriftjava.QueryCompletionEvent thriftEvent = - new com.twitter.presto.thriftjava.QueryCompletionEvent(); - - thriftEvent.query_id = event.getQueryId(); - thriftEvent.transaction_id = event.getTransactionId(); - thriftEvent.user = event.getUser(); - thriftEvent.principal = event.getPrincipal(); - thriftEvent.source = event.getSource(); - thriftEvent.server_version = event.getServerVersion(); - thriftEvent.environment = event.getEnvironment(); - thriftEvent.catalog = event.getCatalog(); - thriftEvent.schema = event.getSchema(); - thriftEvent.remote_client_address = event.getRemoteClientAddress(); - thriftEvent.user_agent = event.getUserAgent(); - thriftEvent.query_state = QueryState.valueOf(event.getQueryState()); - thriftEvent.uri = event.getUri(); - thriftEvent.field_names = event.getFieldNames(); - thriftEvent.query = event.getQuery(); - thriftEvent.create_time_ms = event.getCreateTime().getMillis(); - thriftEvent.execution_start_time_ms = event.getExecutionStartTime().getMillis(); - thriftEvent.end_time_ms = event.getEndTime().getMillis(); - thriftEvent.queued_time_ms = event.getQueuedTimeMs(); - if (event.getAnalysisTimeMs() != null) { - thriftEvent.analysis_time_ms = event.getAnalysisTimeMs(); - } - if (event.getDistributedPlanningTimeMs() != null) { - thriftEvent.distributed_planning_time_ms = event.getDistributedPlanningTimeMs(); - } - if (event.getTotalSplitWallTimeMs() != null) { - thriftEvent.total_split_wall_time_ms = event.getTotalSplitWallTimeMs(); - } - if (event.getTotalSplitCpuTimeMs() != null) { - thriftEvent.total_split_cpu_time_ms = event.getTotalSplitCpuTimeMs(); - } - if (event.getTotalBytes() != null) { - thriftEvent.total_bytes = event.getTotalBytes(); - } - if (event.getTotalRows() != null) { - thriftEvent.total_rows = event.getTotalRows(); - } - thriftEvent.splits = event.getSplits(); - if (event.getErrorCode() != null) { - thriftEvent.error_code_id = event.getErrorCode(); - } - thriftEvent.error_code_name = event.getErrorCodeName(); - thriftEvent.failure_type = event.getFailureType(); - thriftEvent.failure_message = event.getFailureMessage(); - thriftEvent.failure_task = event.getFailureTask(); - thriftEvent.failure_host = event.getFailureHost(); - thriftEvent.output_stage_json = event.getOutputStageJson(); - thriftEvent.failures_json = event.getFailuresJson(); - thriftEvent.inputs_json = event.getInputsJson(); - thriftEvent.session_properties_json = event.getSessionPropertiesJson(); - thriftEvent.query_wall_time_ms = event.getQueryWallTimeMs(); - if (event.getBytesPerSec() != null) { - thriftEvent.bytes_per_sec = event.getBytesPerSec(); - } - if (event.getBytesPerCpuSec() != null) { - thriftEvent.bytes_per_cpu_sec = event.getBytesPerCpuSec(); - } - if (event.getRowsPerSec() != null) { - thriftEvent.rows_per_sec = event.getRowsPerSec(); - } - if (event.getRowsPerCpuSec() != null) { - thriftEvent.rows_per_cpu_sec = event.getRowsPerCpuSec(); - } - - return thriftEvent; - } -} From 77b5d27f2450c1d60b10741e89bbb09c6634820c Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Thu, 1 Dec 2016 14:43:10 -0800 Subject: [PATCH 108/331] upgrade to oss 0.157 without new twitter event scriber --- pom.xml | 5 +- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-server/src/main/provisio/presto.xml | 5 - presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 105 ---------- .../QueryCompletedEventScriber.java | 93 --------- .../plugin/eventlistener/Stringify.java | 183 ------------------ .../eventlistener/TwitterEventListener.java | 44 ----- .../TwitterEventListenerFactory.java | 34 ---- .../TwitterEventListenerPlugin.java | 28 --- .../plugin/eventlistener/TwitterScriber.java | 89 --------- 53 files changed, 46 insertions(+), 628 deletions(-) delete mode 100644 twitter-eventlistener-plugin/pom.xml delete mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java delete mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/Stringify.java delete mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java delete mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerFactory.java delete mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerPlugin.java delete mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java diff --git a/pom.xml b/pom.xml index 77381e4f9179..81e75c58d926 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.157-tw-0.28 + 0.149-tw-0.27 @@ -91,7 +91,6 @@ presto-hive-cdh5 presto-teradata-functions presto-example-http - twitter-eventlistener-plugin presto-local-file presto-tpch presto-raptor diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index c626c3ad70b2..65ba4f1fb785 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index a500d24dd11f..c299b9aca579 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 8761d3638bc6..becb28e3392a 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 73936549aa05..d017a7eb34d9 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 4607b9c36eb8..7e26a4b4544a 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index d7b20168f9d9..c561d30f9e3f 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157-tw-0.28 + 0.149-tw-0.27 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 8ccb6e9fab7a..adf2c74a21e5 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index e5c8a761868a..147e19a1d0be 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index f49684f0083f..7c27715b1043 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index e2e5a57835bf..4ce75a91f29e 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 4f53264a9894..fc675ad793c1 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 0cffa89f962b..14ebeeb13e0c 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 533ffeda6853..7877315a96f6 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 2af0998f777e..5c60a1fe9932 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 03c62da1a4db..814bd89c1b11 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 525324657f8d..518124c6d008 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 7dfdfd71281c..5125111e3813 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 94c8f9f305e7..1ce9320300c8 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 059491bbb6f6..de39cf699043 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 68f97938c645..9ffea9cb6acd 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index e1d4ffd5735a..2094547348f0 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 9c387bfc64c5..9effa7e37a10 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 93da37fd6fc6..094ca0e56546 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 008c82fdbe20..c0d1ebb11fbb 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 7087a7cdab75..41a39c831514 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 874270eb3e39..6fb9a73479e8 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 40c93e82bd6e..30cc3fa661f8 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index a73eefd181a2..4ba0e816b0f5 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index e3e52ccc9c94..2e905d5fdd2c 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 7d6c08ae0f32..75097c126502 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 699d996b9cf9..23952ce91df3 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157-tw-0.28 + 0.149-tw-0.27 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index fc953510fc6e..c7bf601cac62 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index b550cf7cfe49..90ab69762ffb 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index caa9be15c786..6d2f3a67c951 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 33eabb69a5fa..095928ee3535 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 0dec7412e1e5..c265d41f9d53 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index dedf4daf4cb8..202097b9ad4e 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 51b1da095a93..6ceae8a36b78 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-server diff --git a/presto-server/src/main/provisio/presto.xml b/presto-server/src/main/provisio/presto.xml index 584a241f24d7..1c1d17682dda 100644 --- a/presto-server/src/main/provisio/presto.xml +++ b/presto-server/src/main/provisio/presto.xml @@ -152,9 +152,4 @@ - - - - - diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 8bb450c60706..3f61885c321f 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 2b9ec4feb53f..a66a95ad0657 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 8ef0f28df850..46b9e6f1bcc4 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 0923fa884822..5fb15b7fa9ef 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157-tw-0.28 + 0.149-tw-0.27 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 9482bda29427..761874475b86 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 539a9fc3f03f..7646cd0c933f 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml deleted file mode 100644 index a38d41663d1f..000000000000 --- a/twitter-eventlistener-plugin/pom.xml +++ /dev/null @@ -1,105 +0,0 @@ - - - 4.0.0 - - com.facebook.presto - presto-root - 0.157-tw-0.28 - - - twitter-eventlistener-plugin - Twitter Event Listener - scribes QueryCompletedEvent - presto-plugin - - - ${project.parent.basedir} - - - - - com.facebook.presto - presto-spi - 0.157-tw-0.28 - provided - - - io.airlift - log - - - com.google.guava - guava - - - - - com.twitter - presto-thrift-java - 0.0.1 - - - com.twitter - util-core_2.11 - - - com.twitter - util-core-java - - - com.twitter - util-function_2.10 - - - com.twitter - util-function-java - - - commons-logging - commons-logging - - - org.scala-lang.modules - scala-parser-combinators_2.11 - - - com.twitter - scrooge-core - - - org.scala-lang - scala-library - - - org.scala-lang - scala-reflect - - - - - com.twitter - util-logging_2.10 - 6.34.0 - - - commons-logging - commons-logging - - - - - org.apache.thrift - libthrift - - - org.scala-lang - scala-library - 2.10.6 - - - commons-logging - commons-logging - - - - - diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java deleted file mode 100644 index 9075bac1dd8b..000000000000 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java +++ /dev/null @@ -1,93 +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.twitter.presto.plugin.eventlistener; - -import com.facebook.presto.spi.eventlistener.QueryCompletedEvent; -import com.facebook.presto.spi.eventlistener.QueryContext; -import com.facebook.presto.spi.eventlistener.QueryFailureInfo; -import com.facebook.presto.spi.eventlistener.QueryMetadata; -import com.facebook.presto.spi.eventlistener.QueryStatistics; - -import com.twitter.presto.thriftjava.QueryCompletionEvent; -import com.twitter.presto.thriftjava.QueryState; - -/** - * Class that scribes query completion events - */ -public class QueryCompletedEventScriber extends TwitterScriber -{ - public QueryCompletedEventScriber() - { - super("presto_query_completion"); - } - - public void handle(QueryCompletedEvent event) - { - QueryCompletionEvent thriftEvent = toThriftQueryCompletionEvent(event); - if (!scribe(serializeThriftToString(thriftEvent))) { - log.warn("Unable to serialize QueryCompletedEvent: " + event); - } - } - - private static QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletedEvent event) - { - QueryMetadata eventMetadata = event.getMetadata(); - QueryContext eventContext = event.getContext(); - QueryStatistics eventStat = event.getStatistics(); - - QueryCompletionEvent thriftEvent = - new com.twitter.presto.thriftjava.QueryCompletionEvent(); - - thriftEvent.query_id = eventMetadata.getQueryId(); - thriftEvent.transaction_id = eventMetadata.getTransactionId().orElse(DASH); - thriftEvent.user = eventContext.getUser(); - thriftEvent.principal = eventContext.getPrincipal().orElse(DASH); - thriftEvent.source = eventContext.getSource().orElse(DASH); - thriftEvent.server_version = eventContext.getServerVersion(); - thriftEvent.environment = eventContext.getEnvironment(); - thriftEvent.catalog = eventContext.getCatalog().orElse(DASH); - thriftEvent.schema = eventContext.getSchema().orElse(DASH); - thriftEvent.remote_client_address = eventContext.getRemoteClientAddress().orElse(DASH); - thriftEvent.user_agent = eventContext.getUserAgent().orElse(DASH); - thriftEvent.query_state = QueryState.valueOf(eventMetadata.getQueryState()); - thriftEvent.uri = eventMetadata.getUri().toString(); - thriftEvent.query = eventMetadata.getQuery(); - thriftEvent.create_time_ms = event.getCreateTime().toEpochMilli(); - thriftEvent.execution_start_time_ms = event.getExecutionStartTime().toEpochMilli(); - thriftEvent.end_time_ms = event.getEndTime().toEpochMilli(); - thriftEvent.queued_time_ms = eventStat.getQueuedTime().toMillis(); - thriftEvent.query_wall_time_ms = eventStat.getWallTime().toMillis(); - if (eventStat.getAnalysisTime().isPresent()) { - thriftEvent.analysis_time_ms = eventStat.getAnalysisTime().get().toMillis(); - } - if (eventStat.getDistributedPlanningTime().isPresent()) { - thriftEvent.distributed_planning_time_ms = eventStat.getDistributedPlanningTime().get().toMillis(); - } - thriftEvent.total_bytes = eventStat.getTotalBytes(); - thriftEvent.total_rows = eventStat.getTotalRows(); - thriftEvent.splits = eventStat.getCompletedSplits(); - if (event.getFailureInfo().isPresent()) { - QueryFailureInfo eventFailureInfo = event.getFailureInfo().get(); - thriftEvent.error_code_id = eventFailureInfo.getErrorCode().getCode(); - thriftEvent.error_code_name = eventFailureInfo.getErrorCode().getName(); - thriftEvent.failure_type = eventFailureInfo.getFailureType().orElse(DASH); - thriftEvent.failure_message = eventFailureInfo.getFailureMessage().orElse(DASH); - thriftEvent.failure_task = eventFailureInfo.getFailureTask().orElse(DASH); - thriftEvent.failure_host = eventFailureInfo.getFailureHost().orElse(DASH); - thriftEvent.failures_json = eventFailureInfo.getFailuresJson(); - } - - return thriftEvent; - } -} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/Stringify.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/Stringify.java deleted file mode 100644 index 861511ad49ee..000000000000 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/Stringify.java +++ /dev/null @@ -1,183 +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.twitter.presto.plugin.eventlistener; - -import com.facebook.presto.spi.eventlistener.QueryCompletedEvent; -import com.facebook.presto.spi.eventlistener.QueryContext; -import com.facebook.presto.spi.eventlistener.QueryCreatedEvent; -import com.facebook.presto.spi.eventlistener.QueryFailureInfo; -import com.facebook.presto.spi.eventlistener.QueryMetadata; -import com.facebook.presto.spi.eventlistener.QueryStatistics; -import com.facebook.presto.spi.eventlistener.SplitCompletedEvent; -import com.facebook.presto.spi.eventlistener.SplitFailureInfo; -import com.facebook.presto.spi.eventlistener.SplitStatistics; - -import java.time.Duration; -import java.util.Optional; - -class Stringify -{ - private static final String DASH = "-"; - private static final String FIELD_DELIMITER = " ,"; - private static final String OBJECT_DELIMITER = "\n\t"; - - private static String getMillisOrElse(Optional d, String defaultVal) - { - return d.isPresent() ? String.valueOf(d.get().toMillis()) : defaultVal; - } - - private Stringify() - {} - - public static String toString(QueryContext o) - { - String[] content = {"environment(%s)", - "user(%s)", - "userAgent(%s)", - "source(%s)", - "catalog(%s)", - "schema(%s)", - "principal(%s)", - "remoteClientAddress(%s)"}; - return String.format( - String.format("QueryContext: %s", String.join(FIELD_DELIMITER, content)), - o.getEnvironment(), - o.getUser(), - o.getUserAgent().orElse(DASH), - o.getSource().orElse(DASH), - o.getCatalog().orElse(DASH), - o.getSchema().orElse(DASH), - o.getPrincipal().orElse(DASH), - o.getRemoteClientAddress().orElse(DASH)); - } - - public static String toString(QueryFailureInfo o) - { - String[] content = {"errorCode(%s)", - "failureType(%s)", - "failureMessage(%s)", - "failureTask(%s)", - "failureHost(%s)"}; - return String.format( - String.format("QueryFailureInfo: %s", String.join(FIELD_DELIMITER, content)), - o.getErrorCode().toString(), - o.getFailureType().orElse(DASH), - o.getFailureMessage().orElse(DASH), - o.getFailureTask().orElse(DASH), - o.getFailureHost().orElse(DASH)); - } - - public static String toString(QueryMetadata o) - { - String[] content = {"queryId(%s)", - "transactionId(%s)", - "query(%s)", - "queryState(%s)", - "uri(%s)", - "payload(%s)"}; - return String.format( - String.format("QueryMetadata: %s", String.join(FIELD_DELIMITER, content)), - o.getQueryId(), - o.getTransactionId().orElse(DASH), - o.getQuery(), - o.getQueryState(), - o.getUri().toString(), - o.getPayload().orElse(DASH)); - } - - public static String toString(QueryStatistics o) - { - String[] content = {"cpuTime(%d ms)", - "wallTime(%d ms)", - "queuedTime(%d ms)", - "analysisTime(%s ms)", - "distributedPlanningTime(%s ms)", - "peakMemoryBytes(%d)", - "totalBytes(%d)", - "totalRows(%d)", - "completedSplits(%d)", - "complete(%b)"}; - return String.format( - String.format("QueryStatistics: %s", String.join(FIELD_DELIMITER, content)), - o.getCpuTime().toMillis(), - o.getWallTime().toMillis(), - o.getQueuedTime().toMillis(), - getMillisOrElse(o.getAnalysisTime(), DASH), - getMillisOrElse(o.getDistributedPlanningTime(), DASH), - o.getPeakMemoryBytes(), - o.getTotalBytes(), - o.getTotalRows(), - o.getCompletedSplits(), - o.isComplete()); - } - - public static String toString(SplitFailureInfo o) - { - String[] content = {"failureType(%s)", - "failureMessage(%s)"}; - return String.format( - String.format("SplitFailureInfo: %s", String.join(FIELD_DELIMITER, content)), - o.getFailureType(), - o.getFailureMessage()); - } - - public static String toString(SplitStatistics o) - { - String[] content = {"cpuTime(%d ms)", - "wallTime(%d ms)", - "queuedTime(%d ms)", - "userTime(%d ms)", - "completedReadTime(%d ms)", - "completedPositions(%d)", - "completedDataSizeBytes(%d)", - "timeToFirstByte(%s ms)", - "timeToLastByte(%s ms)"}; - return String.format( - String.format("SplitStatistics: %s", String.join(FIELD_DELIMITER, content)), - o.getCpuTime().toMillis(), - o.getWallTime().toMillis(), - o.getQueuedTime().toMillis(), - o.getUserTime().toMillis(), - o.getCompletedReadTime().toMillis(), - o.getCompletedPositions(), - o.getCompletedDataSizeBytes(), - getMillisOrElse(o.getTimeToFirstByte(), DASH), - getMillisOrElse(o.getTimeToLastByte(), DASH)); - } - - public static String toString(QueryCompletedEvent o) - { - String[] content = {toString(o.getMetadata()), - toString(o.getStatistics()), - toString(o.getContext()), - o.getFailureInfo().isPresent() ? toString(o.getFailureInfo().get()) : DASH, - String.format("creationTime: %s", o.getCreateTime().toString()), - String.format("executionStartTime: %s", o.getExecutionStartTime().toString()), - String.format("endTime: %s", o.getEndTime().toString())}; - return String.format("\nQueryCompletedEvent:\n\t%s\n", String.join(OBJECT_DELIMITER, content)); - } - - public static String toString(QueryCreatedEvent o) - { - String[] content = {String.format("createTime: %s", o.getCreateTime().toString()), - toString(o.getContext()), - toString(o.getMetadata())}; - return String.format("\nQueryCreatedEvent:\n\t%s\n", String.join(OBJECT_DELIMITER, content)); - } - - public static String toString(SplitCompletedEvent o) - { - return ""; - } -} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java deleted file mode 100644 index 57bb6d812e22..000000000000 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java +++ /dev/null @@ -1,44 +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.twitter.presto.plugin.eventlistener; - -import com.facebook.presto.spi.eventlistener.EventListener; -import com.facebook.presto.spi.eventlistener.QueryCompletedEvent; -import com.facebook.presto.spi.eventlistener.QueryCreatedEvent; -import com.facebook.presto.spi.eventlistener.SplitCompletedEvent; - -import io.airlift.log.Logger; - -public class TwitterEventListener implements EventListener -{ - private static final Logger log = Logger.get(TwitterEventListener.class); - private final QueryCompletedEventScriber scriber = new QueryCompletedEventScriber(); - - @Override - public void queryCreated(QueryCreatedEvent queryCreatedEvent) - { - } - - @Override - public void queryCompleted(QueryCompletedEvent queryCompletedEvent) - { - log.info(Stringify.toString(queryCompletedEvent)); - scriber.handle(queryCompletedEvent); - } - - @Override - public void splitCompleted(SplitCompletedEvent splitCompletedEvent) - { - } -} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerFactory.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerFactory.java deleted file mode 100644 index 6fedc2d780f1..000000000000 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerFactory.java +++ /dev/null @@ -1,34 +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.twitter.presto.plugin.eventlistener; - -import com.facebook.presto.spi.eventlistener.EventListener; -import com.facebook.presto.spi.eventlistener.EventListenerFactory; - -import java.util.Map; - -public class TwitterEventListenerFactory implements EventListenerFactory -{ - @Override - public String getName() - { - return "twitter-event-listener"; - } - - @Override - public EventListener create(Map config) - { - return new TwitterEventListener(); - } -} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerPlugin.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerPlugin.java deleted file mode 100644 index 557256058490..000000000000 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerPlugin.java +++ /dev/null @@ -1,28 +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.twitter.presto.plugin.eventlistener; - -import com.facebook.presto.spi.Plugin; -import com.facebook.presto.spi.eventlistener.EventListenerFactory; - -import com.google.common.collect.ImmutableList; - -public class TwitterEventListenerPlugin implements Plugin -{ - @Override - public Iterable getEventListenerFactories() - { - return ImmutableList.of(new TwitterEventListenerFactory()); - } -} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java deleted file mode 100644 index 1b7e6714e084..000000000000 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java +++ /dev/null @@ -1,89 +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.twitter.presto.plugin.eventlistener; - -import com.twitter.logging.BareFormatter$; -import com.twitter.logging.Level; -import com.twitter.logging.QueueingHandler; -import com.twitter.logging.ScribeHandler; - -import io.airlift.log.Logger; -import org.apache.thrift.TBase; -import org.apache.thrift.TException; -import org.apache.thrift.TSerializer; - -import java.util.Base64; -import java.util.Optional; -import java.util.logging.LogRecord; - -public class TwitterScriber -{ - protected static final String DASH = "-"; - protected static final int MAX_QUEUE_SIZE = 1000; - - protected static final Logger log = Logger.get(TwitterScriber.class); - - private QueueingHandler queueingHandler; - - // TSerializer is not thread safe - private final ThreadLocal serializer = new ThreadLocal() - { - @Override protected TSerializer initialValue() - { - return new TSerializer(); - } - }; - - public TwitterScriber(String scribeCategory) - { - ScribeHandler scribeHandler = new ScribeHandler( - ScribeHandler.DefaultHostname(), - ScribeHandler.DefaultPort(), - scribeCategory, - ScribeHandler.DefaultBufferTime(), - ScribeHandler.DefaultConnectBackoff(), - ScribeHandler.DefaultMaxMessagesPerTransaction(), - ScribeHandler.DefaultMaxMessagesToBuffer(), - BareFormatter$.MODULE$, - scala.Option.apply((Level) null)); - queueingHandler = new QueueingHandler(scribeHandler, MAX_QUEUE_SIZE); - } - - /** - * Serialize a thrift object to bytes, compress, then encode as a base64 string. - */ - protected Optional serializeThriftToString(TBase thriftMessage) - { - try { - return Optional.of( - Base64.getEncoder().encodeToString(serializer.get().serialize(thriftMessage))); - } - catch (TException e) { - log.warn(e, "Could not serialize thrift object" + thriftMessage); - return Optional.empty(); - } - } - - protected boolean scribe(Optional message) - { - if (message.isPresent()) { - LogRecord logRecord = new LogRecord(Level.ALL, message.get()); - queueingHandler.publish(logRecord); - return true; - } - else { - return false; - } - } -} From c6fe499465417d2fcafa0674caffdc2e61d0619a Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Thu, 1 Dec 2016 18:06:06 -0800 Subject: [PATCH 109/331] add new twitter event scriber impl as plugin --- presto-server/src/main/provisio/presto.xml | 5 + twitter-eventlistener-plugin/pom.xml | 105 ++++++++++ .../QueryCompletedEventScriber.java | 93 +++++++++ .../plugin/eventlistener/Stringify.java | 183 ++++++++++++++++++ .../eventlistener/TwitterEventListener.java | 44 +++++ .../TwitterEventListenerFactory.java | 34 ++++ .../TwitterEventListenerPlugin.java | 28 +++ .../plugin/eventlistener/TwitterScriber.java | 89 +++++++++ 8 files changed, 581 insertions(+) create mode 100644 twitter-eventlistener-plugin/pom.xml create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/Stringify.java create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerFactory.java create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerPlugin.java create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java diff --git a/presto-server/src/main/provisio/presto.xml b/presto-server/src/main/provisio/presto.xml index 1c1d17682dda..584a241f24d7 100644 --- a/presto-server/src/main/provisio/presto.xml +++ b/presto-server/src/main/provisio/presto.xml @@ -152,4 +152,9 @@ + + + + + diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml new file mode 100644 index 000000000000..a38d41663d1f --- /dev/null +++ b/twitter-eventlistener-plugin/pom.xml @@ -0,0 +1,105 @@ + + + 4.0.0 + + com.facebook.presto + presto-root + 0.157-tw-0.28 + + + twitter-eventlistener-plugin + Twitter Event Listener - scribes QueryCompletedEvent + presto-plugin + + + ${project.parent.basedir} + + + + + com.facebook.presto + presto-spi + 0.157-tw-0.28 + provided + + + io.airlift + log + + + com.google.guava + guava + + + + + com.twitter + presto-thrift-java + 0.0.1 + + + com.twitter + util-core_2.11 + + + com.twitter + util-core-java + + + com.twitter + util-function_2.10 + + + com.twitter + util-function-java + + + commons-logging + commons-logging + + + org.scala-lang.modules + scala-parser-combinators_2.11 + + + com.twitter + scrooge-core + + + org.scala-lang + scala-library + + + org.scala-lang + scala-reflect + + + + + com.twitter + util-logging_2.10 + 6.34.0 + + + commons-logging + commons-logging + + + + + org.apache.thrift + libthrift + + + org.scala-lang + scala-library + 2.10.6 + + + commons-logging + commons-logging + + + + + diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java new file mode 100644 index 000000000000..9075bac1dd8b --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java @@ -0,0 +1,93 @@ +/* + * 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.twitter.presto.plugin.eventlistener; + +import com.facebook.presto.spi.eventlistener.QueryCompletedEvent; +import com.facebook.presto.spi.eventlistener.QueryContext; +import com.facebook.presto.spi.eventlistener.QueryFailureInfo; +import com.facebook.presto.spi.eventlistener.QueryMetadata; +import com.facebook.presto.spi.eventlistener.QueryStatistics; + +import com.twitter.presto.thriftjava.QueryCompletionEvent; +import com.twitter.presto.thriftjava.QueryState; + +/** + * Class that scribes query completion events + */ +public class QueryCompletedEventScriber extends TwitterScriber +{ + public QueryCompletedEventScriber() + { + super("presto_query_completion"); + } + + public void handle(QueryCompletedEvent event) + { + QueryCompletionEvent thriftEvent = toThriftQueryCompletionEvent(event); + if (!scribe(serializeThriftToString(thriftEvent))) { + log.warn("Unable to serialize QueryCompletedEvent: " + event); + } + } + + private static QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletedEvent event) + { + QueryMetadata eventMetadata = event.getMetadata(); + QueryContext eventContext = event.getContext(); + QueryStatistics eventStat = event.getStatistics(); + + QueryCompletionEvent thriftEvent = + new com.twitter.presto.thriftjava.QueryCompletionEvent(); + + thriftEvent.query_id = eventMetadata.getQueryId(); + thriftEvent.transaction_id = eventMetadata.getTransactionId().orElse(DASH); + thriftEvent.user = eventContext.getUser(); + thriftEvent.principal = eventContext.getPrincipal().orElse(DASH); + thriftEvent.source = eventContext.getSource().orElse(DASH); + thriftEvent.server_version = eventContext.getServerVersion(); + thriftEvent.environment = eventContext.getEnvironment(); + thriftEvent.catalog = eventContext.getCatalog().orElse(DASH); + thriftEvent.schema = eventContext.getSchema().orElse(DASH); + thriftEvent.remote_client_address = eventContext.getRemoteClientAddress().orElse(DASH); + thriftEvent.user_agent = eventContext.getUserAgent().orElse(DASH); + thriftEvent.query_state = QueryState.valueOf(eventMetadata.getQueryState()); + thriftEvent.uri = eventMetadata.getUri().toString(); + thriftEvent.query = eventMetadata.getQuery(); + thriftEvent.create_time_ms = event.getCreateTime().toEpochMilli(); + thriftEvent.execution_start_time_ms = event.getExecutionStartTime().toEpochMilli(); + thriftEvent.end_time_ms = event.getEndTime().toEpochMilli(); + thriftEvent.queued_time_ms = eventStat.getQueuedTime().toMillis(); + thriftEvent.query_wall_time_ms = eventStat.getWallTime().toMillis(); + if (eventStat.getAnalysisTime().isPresent()) { + thriftEvent.analysis_time_ms = eventStat.getAnalysisTime().get().toMillis(); + } + if (eventStat.getDistributedPlanningTime().isPresent()) { + thriftEvent.distributed_planning_time_ms = eventStat.getDistributedPlanningTime().get().toMillis(); + } + thriftEvent.total_bytes = eventStat.getTotalBytes(); + thriftEvent.total_rows = eventStat.getTotalRows(); + thriftEvent.splits = eventStat.getCompletedSplits(); + if (event.getFailureInfo().isPresent()) { + QueryFailureInfo eventFailureInfo = event.getFailureInfo().get(); + thriftEvent.error_code_id = eventFailureInfo.getErrorCode().getCode(); + thriftEvent.error_code_name = eventFailureInfo.getErrorCode().getName(); + thriftEvent.failure_type = eventFailureInfo.getFailureType().orElse(DASH); + thriftEvent.failure_message = eventFailureInfo.getFailureMessage().orElse(DASH); + thriftEvent.failure_task = eventFailureInfo.getFailureTask().orElse(DASH); + thriftEvent.failure_host = eventFailureInfo.getFailureHost().orElse(DASH); + thriftEvent.failures_json = eventFailureInfo.getFailuresJson(); + } + + return thriftEvent; + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/Stringify.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/Stringify.java new file mode 100644 index 000000000000..861511ad49ee --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/Stringify.java @@ -0,0 +1,183 @@ +/* + * 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.twitter.presto.plugin.eventlistener; + +import com.facebook.presto.spi.eventlistener.QueryCompletedEvent; +import com.facebook.presto.spi.eventlistener.QueryContext; +import com.facebook.presto.spi.eventlistener.QueryCreatedEvent; +import com.facebook.presto.spi.eventlistener.QueryFailureInfo; +import com.facebook.presto.spi.eventlistener.QueryMetadata; +import com.facebook.presto.spi.eventlistener.QueryStatistics; +import com.facebook.presto.spi.eventlistener.SplitCompletedEvent; +import com.facebook.presto.spi.eventlistener.SplitFailureInfo; +import com.facebook.presto.spi.eventlistener.SplitStatistics; + +import java.time.Duration; +import java.util.Optional; + +class Stringify +{ + private static final String DASH = "-"; + private static final String FIELD_DELIMITER = " ,"; + private static final String OBJECT_DELIMITER = "\n\t"; + + private static String getMillisOrElse(Optional d, String defaultVal) + { + return d.isPresent() ? String.valueOf(d.get().toMillis()) : defaultVal; + } + + private Stringify() + {} + + public static String toString(QueryContext o) + { + String[] content = {"environment(%s)", + "user(%s)", + "userAgent(%s)", + "source(%s)", + "catalog(%s)", + "schema(%s)", + "principal(%s)", + "remoteClientAddress(%s)"}; + return String.format( + String.format("QueryContext: %s", String.join(FIELD_DELIMITER, content)), + o.getEnvironment(), + o.getUser(), + o.getUserAgent().orElse(DASH), + o.getSource().orElse(DASH), + o.getCatalog().orElse(DASH), + o.getSchema().orElse(DASH), + o.getPrincipal().orElse(DASH), + o.getRemoteClientAddress().orElse(DASH)); + } + + public static String toString(QueryFailureInfo o) + { + String[] content = {"errorCode(%s)", + "failureType(%s)", + "failureMessage(%s)", + "failureTask(%s)", + "failureHost(%s)"}; + return String.format( + String.format("QueryFailureInfo: %s", String.join(FIELD_DELIMITER, content)), + o.getErrorCode().toString(), + o.getFailureType().orElse(DASH), + o.getFailureMessage().orElse(DASH), + o.getFailureTask().orElse(DASH), + o.getFailureHost().orElse(DASH)); + } + + public static String toString(QueryMetadata o) + { + String[] content = {"queryId(%s)", + "transactionId(%s)", + "query(%s)", + "queryState(%s)", + "uri(%s)", + "payload(%s)"}; + return String.format( + String.format("QueryMetadata: %s", String.join(FIELD_DELIMITER, content)), + o.getQueryId(), + o.getTransactionId().orElse(DASH), + o.getQuery(), + o.getQueryState(), + o.getUri().toString(), + o.getPayload().orElse(DASH)); + } + + public static String toString(QueryStatistics o) + { + String[] content = {"cpuTime(%d ms)", + "wallTime(%d ms)", + "queuedTime(%d ms)", + "analysisTime(%s ms)", + "distributedPlanningTime(%s ms)", + "peakMemoryBytes(%d)", + "totalBytes(%d)", + "totalRows(%d)", + "completedSplits(%d)", + "complete(%b)"}; + return String.format( + String.format("QueryStatistics: %s", String.join(FIELD_DELIMITER, content)), + o.getCpuTime().toMillis(), + o.getWallTime().toMillis(), + o.getQueuedTime().toMillis(), + getMillisOrElse(o.getAnalysisTime(), DASH), + getMillisOrElse(o.getDistributedPlanningTime(), DASH), + o.getPeakMemoryBytes(), + o.getTotalBytes(), + o.getTotalRows(), + o.getCompletedSplits(), + o.isComplete()); + } + + public static String toString(SplitFailureInfo o) + { + String[] content = {"failureType(%s)", + "failureMessage(%s)"}; + return String.format( + String.format("SplitFailureInfo: %s", String.join(FIELD_DELIMITER, content)), + o.getFailureType(), + o.getFailureMessage()); + } + + public static String toString(SplitStatistics o) + { + String[] content = {"cpuTime(%d ms)", + "wallTime(%d ms)", + "queuedTime(%d ms)", + "userTime(%d ms)", + "completedReadTime(%d ms)", + "completedPositions(%d)", + "completedDataSizeBytes(%d)", + "timeToFirstByte(%s ms)", + "timeToLastByte(%s ms)"}; + return String.format( + String.format("SplitStatistics: %s", String.join(FIELD_DELIMITER, content)), + o.getCpuTime().toMillis(), + o.getWallTime().toMillis(), + o.getQueuedTime().toMillis(), + o.getUserTime().toMillis(), + o.getCompletedReadTime().toMillis(), + o.getCompletedPositions(), + o.getCompletedDataSizeBytes(), + getMillisOrElse(o.getTimeToFirstByte(), DASH), + getMillisOrElse(o.getTimeToLastByte(), DASH)); + } + + public static String toString(QueryCompletedEvent o) + { + String[] content = {toString(o.getMetadata()), + toString(o.getStatistics()), + toString(o.getContext()), + o.getFailureInfo().isPresent() ? toString(o.getFailureInfo().get()) : DASH, + String.format("creationTime: %s", o.getCreateTime().toString()), + String.format("executionStartTime: %s", o.getExecutionStartTime().toString()), + String.format("endTime: %s", o.getEndTime().toString())}; + return String.format("\nQueryCompletedEvent:\n\t%s\n", String.join(OBJECT_DELIMITER, content)); + } + + public static String toString(QueryCreatedEvent o) + { + String[] content = {String.format("createTime: %s", o.getCreateTime().toString()), + toString(o.getContext()), + toString(o.getMetadata())}; + return String.format("\nQueryCreatedEvent:\n\t%s\n", String.join(OBJECT_DELIMITER, content)); + } + + public static String toString(SplitCompletedEvent o) + { + return ""; + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java new file mode 100644 index 000000000000..57bb6d812e22 --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java @@ -0,0 +1,44 @@ +/* + * 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.twitter.presto.plugin.eventlistener; + +import com.facebook.presto.spi.eventlistener.EventListener; +import com.facebook.presto.spi.eventlistener.QueryCompletedEvent; +import com.facebook.presto.spi.eventlistener.QueryCreatedEvent; +import com.facebook.presto.spi.eventlistener.SplitCompletedEvent; + +import io.airlift.log.Logger; + +public class TwitterEventListener implements EventListener +{ + private static final Logger log = Logger.get(TwitterEventListener.class); + private final QueryCompletedEventScriber scriber = new QueryCompletedEventScriber(); + + @Override + public void queryCreated(QueryCreatedEvent queryCreatedEvent) + { + } + + @Override + public void queryCompleted(QueryCompletedEvent queryCompletedEvent) + { + log.info(Stringify.toString(queryCompletedEvent)); + scriber.handle(queryCompletedEvent); + } + + @Override + public void splitCompleted(SplitCompletedEvent splitCompletedEvent) + { + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerFactory.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerFactory.java new file mode 100644 index 000000000000..6fedc2d780f1 --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerFactory.java @@ -0,0 +1,34 @@ +/* + * 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.twitter.presto.plugin.eventlistener; + +import com.facebook.presto.spi.eventlistener.EventListener; +import com.facebook.presto.spi.eventlistener.EventListenerFactory; + +import java.util.Map; + +public class TwitterEventListenerFactory implements EventListenerFactory +{ + @Override + public String getName() + { + return "twitter-event-listener"; + } + + @Override + public EventListener create(Map config) + { + return new TwitterEventListener(); + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerPlugin.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerPlugin.java new file mode 100644 index 000000000000..557256058490 --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerPlugin.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.twitter.presto.plugin.eventlistener; + +import com.facebook.presto.spi.Plugin; +import com.facebook.presto.spi.eventlistener.EventListenerFactory; + +import com.google.common.collect.ImmutableList; + +public class TwitterEventListenerPlugin implements Plugin +{ + @Override + public Iterable getEventListenerFactories() + { + return ImmutableList.of(new TwitterEventListenerFactory()); + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java new file mode 100644 index 000000000000..1b7e6714e084 --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java @@ -0,0 +1,89 @@ +/* + * 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.twitter.presto.plugin.eventlistener; + +import com.twitter.logging.BareFormatter$; +import com.twitter.logging.Level; +import com.twitter.logging.QueueingHandler; +import com.twitter.logging.ScribeHandler; + +import io.airlift.log.Logger; +import org.apache.thrift.TBase; +import org.apache.thrift.TException; +import org.apache.thrift.TSerializer; + +import java.util.Base64; +import java.util.Optional; +import java.util.logging.LogRecord; + +public class TwitterScriber +{ + protected static final String DASH = "-"; + protected static final int MAX_QUEUE_SIZE = 1000; + + protected static final Logger log = Logger.get(TwitterScriber.class); + + private QueueingHandler queueingHandler; + + // TSerializer is not thread safe + private final ThreadLocal serializer = new ThreadLocal() + { + @Override protected TSerializer initialValue() + { + return new TSerializer(); + } + }; + + public TwitterScriber(String scribeCategory) + { + ScribeHandler scribeHandler = new ScribeHandler( + ScribeHandler.DefaultHostname(), + ScribeHandler.DefaultPort(), + scribeCategory, + ScribeHandler.DefaultBufferTime(), + ScribeHandler.DefaultConnectBackoff(), + ScribeHandler.DefaultMaxMessagesPerTransaction(), + ScribeHandler.DefaultMaxMessagesToBuffer(), + BareFormatter$.MODULE$, + scala.Option.apply((Level) null)); + queueingHandler = new QueueingHandler(scribeHandler, MAX_QUEUE_SIZE); + } + + /** + * Serialize a thrift object to bytes, compress, then encode as a base64 string. + */ + protected Optional serializeThriftToString(TBase thriftMessage) + { + try { + return Optional.of( + Base64.getEncoder().encodeToString(serializer.get().serialize(thriftMessage))); + } + catch (TException e) { + log.warn(e, "Could not serialize thrift object" + thriftMessage); + return Optional.empty(); + } + } + + protected boolean scribe(Optional message) + { + if (message.isPresent()) { + LogRecord logRecord = new LogRecord(Level.ALL, message.get()); + queueingHandler.publish(logRecord); + return true; + } + else { + return false; + } + } +} From 978c18637cb8f9c570cde096c7e2ccb150843e49 Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Thu, 1 Dec 2016 18:08:09 -0800 Subject: [PATCH 110/331] add twitter-eventlistener-plugin as module to top level pom --- pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/pom.xml b/pom.xml index 81e75c58d926..5212a89f95dd 100644 --- a/pom.xml +++ b/pom.xml @@ -91,6 +91,7 @@ presto-hive-cdh5 presto-teradata-functions presto-example-http + twitter-eventlistener-plugin presto-local-file presto-tpch presto-raptor From 3df6fed5bd9076a60731ddeab57009cd34b8a034 Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Sat, 3 Dec 2016 14:26:59 -0800 Subject: [PATCH 111/331] address Bill's comments --- pom.xml | 4 +- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- .../QueryCompletedEventScriber.java | 41 ++-- .../plugin/eventlistener/Stringify.java | 183 ------------------ .../eventlistener/TwitterEventListener.java | 1 - .../plugin/eventlistener/TwitterScriber.java | 28 ++- 49 files changed, 80 insertions(+), 265 deletions(-) delete mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/Stringify.java diff --git a/pom.xml b/pom.xml index 5212a89f95dd..77381e4f9179 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.149-tw-0.27 + 0.157-tw-0.28 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 65ba4f1fb785..c626c3ad70b2 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index c299b9aca579..a500d24dd11f 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index becb28e3392a..8761d3638bc6 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index d017a7eb34d9..73936549aa05 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 7e26a4b4544a..4607b9c36eb8 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index c561d30f9e3f..d7b20168f9d9 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.149-tw-0.27 + 0.157-tw-0.28 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index adf2c74a21e5..8ccb6e9fab7a 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 147e19a1d0be..e5c8a761868a 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 7c27715b1043..f49684f0083f 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 4ce75a91f29e..e2e5a57835bf 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index fc675ad793c1..4f53264a9894 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 14ebeeb13e0c..0cffa89f962b 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 7877315a96f6..533ffeda6853 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 5c60a1fe9932..2af0998f777e 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 814bd89c1b11..03c62da1a4db 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 518124c6d008..525324657f8d 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 5125111e3813..7dfdfd71281c 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 1ce9320300c8..94c8f9f305e7 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index de39cf699043..059491bbb6f6 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 9ffea9cb6acd..68f97938c645 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 2094547348f0..e1d4ffd5735a 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 9effa7e37a10..9c387bfc64c5 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 094ca0e56546..93da37fd6fc6 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index c0d1ebb11fbb..008c82fdbe20 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 41a39c831514..7087a7cdab75 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 6fb9a73479e8..874270eb3e39 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 30cc3fa661f8..40c93e82bd6e 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 4ba0e816b0f5..a73eefd181a2 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 2e905d5fdd2c..e3e52ccc9c94 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 75097c126502..7d6c08ae0f32 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 23952ce91df3..699d996b9cf9 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.149-tw-0.27 + 0.157-tw-0.28 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index c7bf601cac62..fc953510fc6e 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 90ab69762ffb..b550cf7cfe49 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 6d2f3a67c951..caa9be15c786 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 095928ee3535..33eabb69a5fa 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index c265d41f9d53..0dec7412e1e5 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 202097b9ad4e..dedf4daf4cb8 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 6ceae8a36b78..51b1da095a93 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 3f61885c321f..8bb450c60706 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index a66a95ad0657..2b9ec4feb53f 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 46b9e6f1bcc4..8ef0f28df850 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 5fb15b7fa9ef..0923fa884822 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.149-tw-0.27 + 0.157-tw-0.28 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 761874475b86..9482bda29427 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 7646cd0c933f..539a9fc3f03f 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-verifier diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java index 9075bac1dd8b..97d8decce936 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java @@ -25,19 +25,20 @@ /** * Class that scribes query completion events */ -public class QueryCompletedEventScriber extends TwitterScriber +public class QueryCompletedEventScriber { - public QueryCompletedEventScriber() - { - super("presto_query_completion"); - } + private static String dash = "-"; + private TwitterScriber scriber = new TwitterScriber("presto_query_completion"); public void handle(QueryCompletedEvent event) { - QueryCompletionEvent thriftEvent = toThriftQueryCompletionEvent(event); - if (!scribe(serializeThriftToString(thriftEvent))) { - log.warn("Unable to serialize QueryCompletedEvent: " + event); - } + scriber.scribe(toThriftQueryCompletionEvent(event), + String.format("Query(id=%s, user=%s, env=%s, schema=%s.%s)", + event.getMetadata().getQueryId(), + event.getContext().getUser(), + event.getContext().getEnvironment(), + event.getContext().getCatalog().orElse(dash), + event.getContext().getSchema().orElse(dash))); } private static QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletedEvent event) @@ -50,16 +51,16 @@ private static QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletedE new com.twitter.presto.thriftjava.QueryCompletionEvent(); thriftEvent.query_id = eventMetadata.getQueryId(); - thriftEvent.transaction_id = eventMetadata.getTransactionId().orElse(DASH); + thriftEvent.transaction_id = eventMetadata.getTransactionId().orElse(dash); thriftEvent.user = eventContext.getUser(); - thriftEvent.principal = eventContext.getPrincipal().orElse(DASH); - thriftEvent.source = eventContext.getSource().orElse(DASH); + thriftEvent.principal = eventContext.getPrincipal().orElse(dash); + thriftEvent.source = eventContext.getSource().orElse(dash); thriftEvent.server_version = eventContext.getServerVersion(); thriftEvent.environment = eventContext.getEnvironment(); - thriftEvent.catalog = eventContext.getCatalog().orElse(DASH); - thriftEvent.schema = eventContext.getSchema().orElse(DASH); - thriftEvent.remote_client_address = eventContext.getRemoteClientAddress().orElse(DASH); - thriftEvent.user_agent = eventContext.getUserAgent().orElse(DASH); + thriftEvent.catalog = eventContext.getCatalog().orElse(dash); + thriftEvent.schema = eventContext.getSchema().orElse(dash); + thriftEvent.remote_client_address = eventContext.getRemoteClientAddress().orElse(dash); + thriftEvent.user_agent = eventContext.getUserAgent().orElse(dash); thriftEvent.query_state = QueryState.valueOf(eventMetadata.getQueryState()); thriftEvent.uri = eventMetadata.getUri().toString(); thriftEvent.query = eventMetadata.getQuery(); @@ -81,10 +82,10 @@ private static QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletedE QueryFailureInfo eventFailureInfo = event.getFailureInfo().get(); thriftEvent.error_code_id = eventFailureInfo.getErrorCode().getCode(); thriftEvent.error_code_name = eventFailureInfo.getErrorCode().getName(); - thriftEvent.failure_type = eventFailureInfo.getFailureType().orElse(DASH); - thriftEvent.failure_message = eventFailureInfo.getFailureMessage().orElse(DASH); - thriftEvent.failure_task = eventFailureInfo.getFailureTask().orElse(DASH); - thriftEvent.failure_host = eventFailureInfo.getFailureHost().orElse(DASH); + thriftEvent.failure_type = eventFailureInfo.getFailureType().orElse(dash); + thriftEvent.failure_message = eventFailureInfo.getFailureMessage().orElse(dash); + thriftEvent.failure_task = eventFailureInfo.getFailureTask().orElse(dash); + thriftEvent.failure_host = eventFailureInfo.getFailureHost().orElse(dash); thriftEvent.failures_json = eventFailureInfo.getFailuresJson(); } diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/Stringify.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/Stringify.java deleted file mode 100644 index 861511ad49ee..000000000000 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/Stringify.java +++ /dev/null @@ -1,183 +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.twitter.presto.plugin.eventlistener; - -import com.facebook.presto.spi.eventlistener.QueryCompletedEvent; -import com.facebook.presto.spi.eventlistener.QueryContext; -import com.facebook.presto.spi.eventlistener.QueryCreatedEvent; -import com.facebook.presto.spi.eventlistener.QueryFailureInfo; -import com.facebook.presto.spi.eventlistener.QueryMetadata; -import com.facebook.presto.spi.eventlistener.QueryStatistics; -import com.facebook.presto.spi.eventlistener.SplitCompletedEvent; -import com.facebook.presto.spi.eventlistener.SplitFailureInfo; -import com.facebook.presto.spi.eventlistener.SplitStatistics; - -import java.time.Duration; -import java.util.Optional; - -class Stringify -{ - private static final String DASH = "-"; - private static final String FIELD_DELIMITER = " ,"; - private static final String OBJECT_DELIMITER = "\n\t"; - - private static String getMillisOrElse(Optional d, String defaultVal) - { - return d.isPresent() ? String.valueOf(d.get().toMillis()) : defaultVal; - } - - private Stringify() - {} - - public static String toString(QueryContext o) - { - String[] content = {"environment(%s)", - "user(%s)", - "userAgent(%s)", - "source(%s)", - "catalog(%s)", - "schema(%s)", - "principal(%s)", - "remoteClientAddress(%s)"}; - return String.format( - String.format("QueryContext: %s", String.join(FIELD_DELIMITER, content)), - o.getEnvironment(), - o.getUser(), - o.getUserAgent().orElse(DASH), - o.getSource().orElse(DASH), - o.getCatalog().orElse(DASH), - o.getSchema().orElse(DASH), - o.getPrincipal().orElse(DASH), - o.getRemoteClientAddress().orElse(DASH)); - } - - public static String toString(QueryFailureInfo o) - { - String[] content = {"errorCode(%s)", - "failureType(%s)", - "failureMessage(%s)", - "failureTask(%s)", - "failureHost(%s)"}; - return String.format( - String.format("QueryFailureInfo: %s", String.join(FIELD_DELIMITER, content)), - o.getErrorCode().toString(), - o.getFailureType().orElse(DASH), - o.getFailureMessage().orElse(DASH), - o.getFailureTask().orElse(DASH), - o.getFailureHost().orElse(DASH)); - } - - public static String toString(QueryMetadata o) - { - String[] content = {"queryId(%s)", - "transactionId(%s)", - "query(%s)", - "queryState(%s)", - "uri(%s)", - "payload(%s)"}; - return String.format( - String.format("QueryMetadata: %s", String.join(FIELD_DELIMITER, content)), - o.getQueryId(), - o.getTransactionId().orElse(DASH), - o.getQuery(), - o.getQueryState(), - o.getUri().toString(), - o.getPayload().orElse(DASH)); - } - - public static String toString(QueryStatistics o) - { - String[] content = {"cpuTime(%d ms)", - "wallTime(%d ms)", - "queuedTime(%d ms)", - "analysisTime(%s ms)", - "distributedPlanningTime(%s ms)", - "peakMemoryBytes(%d)", - "totalBytes(%d)", - "totalRows(%d)", - "completedSplits(%d)", - "complete(%b)"}; - return String.format( - String.format("QueryStatistics: %s", String.join(FIELD_DELIMITER, content)), - o.getCpuTime().toMillis(), - o.getWallTime().toMillis(), - o.getQueuedTime().toMillis(), - getMillisOrElse(o.getAnalysisTime(), DASH), - getMillisOrElse(o.getDistributedPlanningTime(), DASH), - o.getPeakMemoryBytes(), - o.getTotalBytes(), - o.getTotalRows(), - o.getCompletedSplits(), - o.isComplete()); - } - - public static String toString(SplitFailureInfo o) - { - String[] content = {"failureType(%s)", - "failureMessage(%s)"}; - return String.format( - String.format("SplitFailureInfo: %s", String.join(FIELD_DELIMITER, content)), - o.getFailureType(), - o.getFailureMessage()); - } - - public static String toString(SplitStatistics o) - { - String[] content = {"cpuTime(%d ms)", - "wallTime(%d ms)", - "queuedTime(%d ms)", - "userTime(%d ms)", - "completedReadTime(%d ms)", - "completedPositions(%d)", - "completedDataSizeBytes(%d)", - "timeToFirstByte(%s ms)", - "timeToLastByte(%s ms)"}; - return String.format( - String.format("SplitStatistics: %s", String.join(FIELD_DELIMITER, content)), - o.getCpuTime().toMillis(), - o.getWallTime().toMillis(), - o.getQueuedTime().toMillis(), - o.getUserTime().toMillis(), - o.getCompletedReadTime().toMillis(), - o.getCompletedPositions(), - o.getCompletedDataSizeBytes(), - getMillisOrElse(o.getTimeToFirstByte(), DASH), - getMillisOrElse(o.getTimeToLastByte(), DASH)); - } - - public static String toString(QueryCompletedEvent o) - { - String[] content = {toString(o.getMetadata()), - toString(o.getStatistics()), - toString(o.getContext()), - o.getFailureInfo().isPresent() ? toString(o.getFailureInfo().get()) : DASH, - String.format("creationTime: %s", o.getCreateTime().toString()), - String.format("executionStartTime: %s", o.getExecutionStartTime().toString()), - String.format("endTime: %s", o.getEndTime().toString())}; - return String.format("\nQueryCompletedEvent:\n\t%s\n", String.join(OBJECT_DELIMITER, content)); - } - - public static String toString(QueryCreatedEvent o) - { - String[] content = {String.format("createTime: %s", o.getCreateTime().toString()), - toString(o.getContext()), - toString(o.getMetadata())}; - return String.format("\nQueryCreatedEvent:\n\t%s\n", String.join(OBJECT_DELIMITER, content)); - } - - public static String toString(SplitCompletedEvent o) - { - return ""; - } -} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java index 57bb6d812e22..0c622b0ba34c 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java @@ -33,7 +33,6 @@ public void queryCreated(QueryCreatedEvent queryCreatedEvent) @Override public void queryCompleted(QueryCompletedEvent queryCompletedEvent) { - log.info(Stringify.toString(queryCompletedEvent)); scriber.handle(queryCompletedEvent); } diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java index 1b7e6714e084..11be68af43d2 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java @@ -24,7 +24,6 @@ import org.apache.thrift.TSerializer; import java.util.Base64; -import java.util.Optional; import java.util.logging.LogRecord; public class TwitterScriber @@ -62,28 +61,27 @@ public TwitterScriber(String scribeCategory) /** * Serialize a thrift object to bytes, compress, then encode as a base64 string. + * Throws TException */ - protected Optional serializeThriftToString(TBase thriftMessage) + public String serializeThriftToString(TBase thriftMessage) throws TException + { + return Base64.getEncoder().encodeToString(serializer.get().serialize(thriftMessage)); + } + + public void scribe(TBase thriftMessage, String origEventIdentifier) { try { - return Optional.of( - Base64.getEncoder().encodeToString(serializer.get().serialize(thriftMessage))); + String encodedStr = serializeThriftToString(thriftMessage); + scribe(encodedStr); } catch (TException e) { - log.warn(e, "Could not serialize thrift object" + thriftMessage); - return Optional.empty(); + log.warn(e, String.format("Could not serialize thrift object of %s", origEventIdentifier)); } } - protected boolean scribe(Optional message) + public void scribe(String message) { - if (message.isPresent()) { - LogRecord logRecord = new LogRecord(Level.ALL, message.get()); - queueingHandler.publish(logRecord); - return true; - } - else { - return false; - } + LogRecord logRecord = new LogRecord(Level.ALL, message); + queueingHandler.publish(logRecord); } } From 86917eaa24ad4d7867460d24b1db5a813c7045b8 Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Mon, 5 Dec 2016 12:16:25 -0800 Subject: [PATCH 112/331] back out version change, address comments --- pom.xml | 4 +- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 +- .../QueryCompletedEventScriber.java | 48 +++++++++++-------- .../plugin/eventlistener/TwitterScriber.java | 48 ++++++++----------- 48 files changed, 97 insertions(+), 95 deletions(-) diff --git a/pom.xml b/pom.xml index 77381e4f9179..5212a89f95dd 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.157-tw-0.28 + 0.149-tw-0.27 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index c626c3ad70b2..65ba4f1fb785 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index a500d24dd11f..c299b9aca579 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 8761d3638bc6..becb28e3392a 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 73936549aa05..d017a7eb34d9 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 4607b9c36eb8..7e26a4b4544a 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index d7b20168f9d9..c561d30f9e3f 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157-tw-0.28 + 0.149-tw-0.27 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 8ccb6e9fab7a..adf2c74a21e5 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index e5c8a761868a..147e19a1d0be 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index f49684f0083f..7c27715b1043 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index e2e5a57835bf..4ce75a91f29e 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 4f53264a9894..fc675ad793c1 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 0cffa89f962b..14ebeeb13e0c 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 533ffeda6853..7877315a96f6 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 2af0998f777e..5c60a1fe9932 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 03c62da1a4db..814bd89c1b11 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 525324657f8d..518124c6d008 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 7dfdfd71281c..5125111e3813 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 94c8f9f305e7..1ce9320300c8 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 059491bbb6f6..de39cf699043 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 68f97938c645..9ffea9cb6acd 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index e1d4ffd5735a..2094547348f0 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 9c387bfc64c5..9effa7e37a10 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 93da37fd6fc6..094ca0e56546 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 008c82fdbe20..c0d1ebb11fbb 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 7087a7cdab75..41a39c831514 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 874270eb3e39..6fb9a73479e8 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 40c93e82bd6e..30cc3fa661f8 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index a73eefd181a2..4ba0e816b0f5 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index e3e52ccc9c94..2e905d5fdd2c 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 7d6c08ae0f32..75097c126502 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 699d996b9cf9..23952ce91df3 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157-tw-0.28 + 0.149-tw-0.27 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index fc953510fc6e..c7bf601cac62 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index b550cf7cfe49..90ab69762ffb 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index caa9be15c786..6d2f3a67c951 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 33eabb69a5fa..095928ee3535 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 0dec7412e1e5..c265d41f9d53 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index dedf4daf4cb8..202097b9ad4e 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 51b1da095a93..6ceae8a36b78 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 8bb450c60706..3f61885c321f 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 2b9ec4feb53f..a66a95ad0657 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 8ef0f28df850..46b9e6f1bcc4 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 0923fa884822..5fb15b7fa9ef 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157-tw-0.28 + 0.149-tw-0.27 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 9482bda29427..761874475b86 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 539a9fc3f03f..7646cd0c933f 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index a38d41663d1f..4ad84ca03757 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.149-tw-0.27 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.157-tw-0.28 + 0.149-tw-0.27 provided diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java index 97d8decce936..e9c2faf3eda0 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java @@ -22,23 +22,33 @@ import com.twitter.presto.thriftjava.QueryCompletionEvent; import com.twitter.presto.thriftjava.QueryState; +import io.airlift.log.Logger; +import org.apache.thrift.TException; + /** * Class that scribes query completion events */ public class QueryCompletedEventScriber { - private static String dash = "-"; + private static final String DASH = "-"; + private static final Logger log = Logger.get(QueryCompletedEventScriber.class); + private TwitterScriber scriber = new TwitterScriber("presto_query_completion"); public void handle(QueryCompletedEvent event) { - scriber.scribe(toThriftQueryCompletionEvent(event), - String.format("Query(id=%s, user=%s, env=%s, schema=%s.%s)", - event.getMetadata().getQueryId(), - event.getContext().getUser(), - event.getContext().getEnvironment(), - event.getContext().getCatalog().orElse(dash), - event.getContext().getSchema().orElse(dash))); + try { + scriber.scribe(toThriftQueryCompletionEvent(event)); + } + catch (TException e) { + log.warn(e, + String.format("Could not serialize thrift object of Query(id=%s, user=%s, env=%s, schema=%s.%s)", + event.getMetadata().getQueryId(), + event.getContext().getUser(), + event.getContext().getEnvironment(), + event.getContext().getCatalog().orElse(DASH), + event.getContext().getSchema().orElse(DASH))); + } } private static QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletedEvent event) @@ -51,16 +61,16 @@ private static QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletedE new com.twitter.presto.thriftjava.QueryCompletionEvent(); thriftEvent.query_id = eventMetadata.getQueryId(); - thriftEvent.transaction_id = eventMetadata.getTransactionId().orElse(dash); + thriftEvent.transaction_id = eventMetadata.getTransactionId().orElse(DASH); thriftEvent.user = eventContext.getUser(); - thriftEvent.principal = eventContext.getPrincipal().orElse(dash); - thriftEvent.source = eventContext.getSource().orElse(dash); + thriftEvent.principal = eventContext.getPrincipal().orElse(DASH); + thriftEvent.source = eventContext.getSource().orElse(DASH); thriftEvent.server_version = eventContext.getServerVersion(); thriftEvent.environment = eventContext.getEnvironment(); - thriftEvent.catalog = eventContext.getCatalog().orElse(dash); - thriftEvent.schema = eventContext.getSchema().orElse(dash); - thriftEvent.remote_client_address = eventContext.getRemoteClientAddress().orElse(dash); - thriftEvent.user_agent = eventContext.getUserAgent().orElse(dash); + thriftEvent.catalog = eventContext.getCatalog().orElse(DASH); + thriftEvent.schema = eventContext.getSchema().orElse(DASH); + thriftEvent.remote_client_address = eventContext.getRemoteClientAddress().orElse(DASH); + thriftEvent.user_agent = eventContext.getUserAgent().orElse(DASH); thriftEvent.query_state = QueryState.valueOf(eventMetadata.getQueryState()); thriftEvent.uri = eventMetadata.getUri().toString(); thriftEvent.query = eventMetadata.getQuery(); @@ -82,10 +92,10 @@ private static QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletedE QueryFailureInfo eventFailureInfo = event.getFailureInfo().get(); thriftEvent.error_code_id = eventFailureInfo.getErrorCode().getCode(); thriftEvent.error_code_name = eventFailureInfo.getErrorCode().getName(); - thriftEvent.failure_type = eventFailureInfo.getFailureType().orElse(dash); - thriftEvent.failure_message = eventFailureInfo.getFailureMessage().orElse(dash); - thriftEvent.failure_task = eventFailureInfo.getFailureTask().orElse(dash); - thriftEvent.failure_host = eventFailureInfo.getFailureHost().orElse(dash); + thriftEvent.failure_type = eventFailureInfo.getFailureType().orElse(DASH); + thriftEvent.failure_message = eventFailureInfo.getFailureMessage().orElse(DASH); + thriftEvent.failure_task = eventFailureInfo.getFailureTask().orElse(DASH); + thriftEvent.failure_host = eventFailureInfo.getFailureHost().orElse(DASH); thriftEvent.failures_json = eventFailureInfo.getFailuresJson(); } diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java index 11be68af43d2..604ebad694e7 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java @@ -18,7 +18,6 @@ import com.twitter.logging.QueueingHandler; import com.twitter.logging.ScribeHandler; -import io.airlift.log.Logger; import org.apache.thrift.TBase; import org.apache.thrift.TException; import org.apache.thrift.TSerializer; @@ -28,10 +27,8 @@ public class TwitterScriber { - protected static final String DASH = "-"; - protected static final int MAX_QUEUE_SIZE = 1000; - - protected static final Logger log = Logger.get(TwitterScriber.class); + private static final String DASH = "-"; + private static final int MAX_QUEUE_SIZE = 1000; private QueueingHandler queueingHandler; @@ -44,6 +41,21 @@ public class TwitterScriber } }; + /** + * Serialize a thrift object to bytes, compress, then encode as a base64 string. + * Throws TException + */ + private String serializeThriftToString(TBase thriftMessage) throws TException + { + return Base64.getEncoder().encodeToString(serializer.get().serialize(thriftMessage)); + } + + private void scribe(String message) + { + LogRecord logRecord = new LogRecord(Level.ALL, message); + queueingHandler.publish(logRecord); + } + public TwitterScriber(String scribeCategory) { ScribeHandler scribeHandler = new ScribeHandler( @@ -59,29 +71,9 @@ public TwitterScriber(String scribeCategory) queueingHandler = new QueueingHandler(scribeHandler, MAX_QUEUE_SIZE); } - /** - * Serialize a thrift object to bytes, compress, then encode as a base64 string. - * Throws TException - */ - public String serializeThriftToString(TBase thriftMessage) throws TException - { - return Base64.getEncoder().encodeToString(serializer.get().serialize(thriftMessage)); - } - - public void scribe(TBase thriftMessage, String origEventIdentifier) + public void scribe(TBase thriftMessage) throws TException { - try { - String encodedStr = serializeThriftToString(thriftMessage); - scribe(encodedStr); - } - catch (TException e) { - log.warn(e, String.format("Could not serialize thrift object of %s", origEventIdentifier)); - } - } - - public void scribe(String message) - { - LogRecord logRecord = new LogRecord(Level.ALL, message); - queueingHandler.publish(logRecord); + String encodedStr = serializeThriftToString(thriftMessage); + scribe(encodedStr); } } From 1d36d6492205047127464905110f97a408887a62 Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Mon, 5 Dec 2016 13:11:56 -0800 Subject: [PATCH 113/331] constructor first then public, private methods --- .../twitter/presto/plugin/eventlistener/:q | 78 +++++++++++++++++++ .../plugin/eventlistener/TwitterScriber.java | 33 ++++---- 2 files changed, 94 insertions(+), 17 deletions(-) create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/:q diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/:q b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/:q new file mode 100644 index 000000000000..9e8524602255 --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/:q @@ -0,0 +1,78 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.twitter.presto.plugin.eventlistener; + +import com.twitter.logging.BareFormatter$; +import com.twitter.logging.Level; +import com.twitter.logging.QueueingHandler; +import com.twitter.logging.ScribeHandler; + +import org.apache.thrift.TBase; +import org.apache.thrift.TException; +import org.apache.thrift.TSerializer; + +import java.util.Base64; +import java.util.logging.LogRecord; + +public class TwitterScriber +{ + private static final String DASH = "-"; + private static final int MAX_QUEUE_SIZE = 1000; + + private QueueingHandler queueingHandler; + + // TSerializer is not thread safe + private final ThreadLocal serializer = new ThreadLocal() + { + @Override protected TSerializer initialValue() + { + return new TSerializer(); + } + }; + + public TwitterScriber(String scribeCategory) + { + ScribeHandler scribeHandler = new ScribeHandler( + ScribeHandler.DefaultHostname(), + ScribeHandler.DefaultPort(), + scribeCategory, + ScribeHandler.DefaultBufferTime(), + ScribeHandler.DefaultConnectBackoff(), + ScribeHandler.DefaultMaxMessagesPerTransaction(), + ScribeHandler.DefaultMaxMessagesToBuffer(), + BareFormatter$.MODULE$, + scala.Option.apply((Level) null)); + queueingHandler = new QueueingHandler(scribeHandler, MAX_QUEUE_SIZE); + } + + public void scribe(TBase thriftMessage) throws TException + { + scribe(serializeThriftToString(thriftMessage)); + } + + /** + * Serialize a thrift object to bytes, compress, then encode as a base64 string. + * Throws TException + */ + private String serializeThriftToString(TBase thriftMessage) throws TException + { + return Base64.getEncoder().encodeToString(serializer.get().serialize(thriftMessage)); + } + + private void scribe(String message) + { + LogRecord logRecord = new LogRecord(Level.ALL, message); + queueingHandler.publish(logRecord); + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java index 604ebad694e7..9e8524602255 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java @@ -41,21 +41,6 @@ public class TwitterScriber } }; - /** - * Serialize a thrift object to bytes, compress, then encode as a base64 string. - * Throws TException - */ - private String serializeThriftToString(TBase thriftMessage) throws TException - { - return Base64.getEncoder().encodeToString(serializer.get().serialize(thriftMessage)); - } - - private void scribe(String message) - { - LogRecord logRecord = new LogRecord(Level.ALL, message); - queueingHandler.publish(logRecord); - } - public TwitterScriber(String scribeCategory) { ScribeHandler scribeHandler = new ScribeHandler( @@ -73,7 +58,21 @@ public TwitterScriber(String scribeCategory) public void scribe(TBase thriftMessage) throws TException { - String encodedStr = serializeThriftToString(thriftMessage); - scribe(encodedStr); + scribe(serializeThriftToString(thriftMessage)); + } + + /** + * Serialize a thrift object to bytes, compress, then encode as a base64 string. + * Throws TException + */ + private String serializeThriftToString(TBase thriftMessage) throws TException + { + return Base64.getEncoder().encodeToString(serializer.get().serialize(thriftMessage)); + } + + private void scribe(String message) + { + LogRecord logRecord = new LogRecord(Level.ALL, message); + queueingHandler.publish(logRecord); } } From 41a0ab334fcbe81ca4f258d2ec27dd5aef988df3 Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Mon, 5 Dec 2016 13:26:12 -0800 Subject: [PATCH 114/331] remove accidental commit --- .../twitter/presto/plugin/eventlistener/:q | 78 ------------------- 1 file changed, 78 deletions(-) delete mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/:q diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/:q b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/:q deleted file mode 100644 index 9e8524602255..000000000000 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/:q +++ /dev/null @@ -1,78 +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.twitter.presto.plugin.eventlistener; - -import com.twitter.logging.BareFormatter$; -import com.twitter.logging.Level; -import com.twitter.logging.QueueingHandler; -import com.twitter.logging.ScribeHandler; - -import org.apache.thrift.TBase; -import org.apache.thrift.TException; -import org.apache.thrift.TSerializer; - -import java.util.Base64; -import java.util.logging.LogRecord; - -public class TwitterScriber -{ - private static final String DASH = "-"; - private static final int MAX_QUEUE_SIZE = 1000; - - private QueueingHandler queueingHandler; - - // TSerializer is not thread safe - private final ThreadLocal serializer = new ThreadLocal() - { - @Override protected TSerializer initialValue() - { - return new TSerializer(); - } - }; - - public TwitterScriber(String scribeCategory) - { - ScribeHandler scribeHandler = new ScribeHandler( - ScribeHandler.DefaultHostname(), - ScribeHandler.DefaultPort(), - scribeCategory, - ScribeHandler.DefaultBufferTime(), - ScribeHandler.DefaultConnectBackoff(), - ScribeHandler.DefaultMaxMessagesPerTransaction(), - ScribeHandler.DefaultMaxMessagesToBuffer(), - BareFormatter$.MODULE$, - scala.Option.apply((Level) null)); - queueingHandler = new QueueingHandler(scribeHandler, MAX_QUEUE_SIZE); - } - - public void scribe(TBase thriftMessage) throws TException - { - scribe(serializeThriftToString(thriftMessage)); - } - - /** - * Serialize a thrift object to bytes, compress, then encode as a base64 string. - * Throws TException - */ - private String serializeThriftToString(TBase thriftMessage) throws TException - { - return Base64.getEncoder().encodeToString(serializer.get().serialize(thriftMessage)); - } - - private void scribe(String message) - { - LogRecord logRecord = new LogRecord(Level.ALL, message); - queueingHandler.publish(logRecord); - } -} From 0ed477adad4f12910b37edefc985b9f6afcab188 Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Mon, 5 Dec 2016 16:48:43 -0800 Subject: [PATCH 115/331] Prepare for 0.157-tw-0.28 internal release --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 46 files changed, 48 insertions(+), 48 deletions(-) diff --git a/pom.xml b/pom.xml index 5212a89f95dd..77381e4f9179 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.149-tw-0.27 + 0.157-tw-0.28 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 65ba4f1fb785..c626c3ad70b2 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index c299b9aca579..a500d24dd11f 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index becb28e3392a..8761d3638bc6 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index d017a7eb34d9..73936549aa05 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 7e26a4b4544a..4607b9c36eb8 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index c561d30f9e3f..d7b20168f9d9 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.149-tw-0.27 + 0.157-tw-0.28 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index adf2c74a21e5..8ccb6e9fab7a 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 147e19a1d0be..e5c8a761868a 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 7c27715b1043..f49684f0083f 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 4ce75a91f29e..e2e5a57835bf 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index fc675ad793c1..4f53264a9894 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 14ebeeb13e0c..0cffa89f962b 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 7877315a96f6..533ffeda6853 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 5c60a1fe9932..2af0998f777e 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 814bd89c1b11..03c62da1a4db 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 518124c6d008..525324657f8d 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 5125111e3813..7dfdfd71281c 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 1ce9320300c8..94c8f9f305e7 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index de39cf699043..059491bbb6f6 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 9ffea9cb6acd..68f97938c645 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 2094547348f0..e1d4ffd5735a 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 9effa7e37a10..9c387bfc64c5 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 094ca0e56546..93da37fd6fc6 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index c0d1ebb11fbb..008c82fdbe20 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 41a39c831514..7087a7cdab75 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 6fb9a73479e8..874270eb3e39 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 30cc3fa661f8..40c93e82bd6e 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 4ba0e816b0f5..a73eefd181a2 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 2e905d5fdd2c..e3e52ccc9c94 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 75097c126502..7d6c08ae0f32 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 23952ce91df3..699d996b9cf9 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.149-tw-0.27 + 0.157-tw-0.28 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index c7bf601cac62..fc953510fc6e 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 90ab69762ffb..b550cf7cfe49 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 6d2f3a67c951..caa9be15c786 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 095928ee3535..33eabb69a5fa 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index c265d41f9d53..0dec7412e1e5 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 202097b9ad4e..dedf4daf4cb8 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 6ceae8a36b78..51b1da095a93 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 3f61885c321f..8bb450c60706 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index a66a95ad0657..2b9ec4feb53f 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 46b9e6f1bcc4..8ef0f28df850 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 5fb15b7fa9ef..0923fa884822 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.149-tw-0.27 + 0.157-tw-0.28 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 761874475b86..9482bda29427 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 7646cd0c933f..539a9fc3f03f 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 4ad84ca03757..a38d41663d1f 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.149-tw-0.27 + 0.157-tw-0.28 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.149-tw-0.27 + 0.157-tw-0.28 provided From c0ada2b05adf2c3c363f6a1b983527863b9be07b Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 7 Dec 2016 17:31:52 -0800 Subject: [PATCH 116/331] Added TTransportPool --- .../PooledHiveMetastoreClientFactory.java | 67 +++++++++++ .../hive/util/PooledTTransportFactory.java | 38 ++++++ .../twitter/hive/util/TTransportPool.java | 108 ++++++++++++++++++ 3 files changed, 213 insertions(+) create mode 100644 presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java new file mode 100644 index 000000000000..cea665466ff1 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java @@ -0,0 +1,67 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.twitter.hive; + +import com.facebook.presto.hive.HiveClientConfig; +import com.facebook.presto.hive.ThriftHiveMetastoreClient; +import com.facebook.presto.hive.authentication.HiveMetastoreAuthentication; +import com.facebook.presto.hive.metastore.HiveMetastoreClient; +import com.facebook.presto.twitter.hive.util.PooledTTransportFactory; +import com.facebook.presto.twitter.hive.util.TTransportPool; +import com.google.common.net.HostAndPort; +import com.google.common.primitives.Ints; +import io.airlift.units.Duration; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; + +import javax.annotation.Nullable; +import javax.inject.Inject; + +import static java.util.Objects.requireNonNull; + +public class PooledHiveMetastoreClientFactory +{ + private final HostAndPort socksProxy; + private final int timeoutMillis; + private final HiveMetastoreAuthentication metastoreAuthentication; + + public PooledHiveMetastoreClientFactory(@Nullable HostAndPort socksProxy, Duration timeout, HiveMetastoreAuthentication metastoreAuthentication) + { + this.socksProxy = socksProxy; + this.timeoutMillis = Ints.checkedCast(timeout.toMillis()); + this.metastoreAuthentication = requireNonNull(metastoreAuthentication, "metastoreAuthentication is null"); + this.transportPool = new TTransportPool(); + } + + @Inject + public PooledHiveMetastoreClientFactory(HiveClientConfig config, HiveMetastoreAuthentication metastoreAuthentication) + { + this(config.getMetastoreSocksProxy(), config.getMetastoreTimeout(), metastoreAuthentication); + } + + public HiveMetastoreClient create(String host, int port) + throws TTransportException + { + try { + TTransport transport= transportPool.borrowObject(host, port); + if (transport == null) { + transport = transportPool.borrowObject(host, port, new PooledTTransportFactory(host, port, socksProxy, timeoutMillis, metastoreAuthentication)); + } + } + catch (Exception e) { + throw new TTransportException(e.getType(), String.format("%s: %s", host, e.getMessage()), e.getCause()) + } + return new ThriftHiveMetastoreClient(transport); + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java new file mode 100644 index 000000000000..5e04ed9f64a6 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java @@ -0,0 +1,38 @@ +/* + * 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.twitter.hive.util; + +import com.facebook.presto.hive.authentication.HiveMetastoreAuthentication; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.net.InetSocketAddress +import java.net.SocketAddress; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +import org.apache.commons.pool2.ObjectPool; +import org.apache.commons.pool2.impl.GenericObjectPool; +import org.apache.thrift.transport.TSocket; +import org.apache.thrift.transport.TTransport; + +/** + * Utility class to handle creating and caching the UserGroupInformation object. + */ +public class PooledTTransportFactory +{ + +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java new file mode 100644 index 000000000000..b148404a7ff6 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java @@ -0,0 +1,108 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.twitter.hive.util; + +import com.facebook.presto.hive.authentication.HiveMetastoreAuthentication; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.net.InetSocketAddress +import java.net.SocketAddress; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +import org.apache.commons.pool2.ObjectPool; +import org.apache.commons.pool2.impl.GenericObjectPool; +import org.apache.thrift.transport.TSocket; +import org.apache.thrift.transport.TTransport; + +/** + * Utility class to handle creating and caching the UserGroupInformation object. + */ +public class TTransportPool +{ + private final ConcurrentMap> pools = new ConcurrentHashMap(); + + public TTransportPool(){} + + private void add(SocketAddress remote, PooledObjectFactory transportFactory) + { + pools.put(remote, new GenericObjectPool(transportFactory)); + } + + protected TTransport get(SocketAddress remote, PooledObjectFactory transportFactory) + { + ObjectPool pool = pools.get(remote) + if (pool == null) + { + add(remote, transportFactory); + pool = pools.get(remote); + } + + return pool.borrowObject(); + } + + protected TTransport get(SocketAddress remote) + { + ObjectPool pool = pools.get(remote) + if (pool == null) + { + return null; + } + + return pool.borrowObject(); + } + + public TTransport borrowObject(String host, int port, PooledObjectFactory transportFactory) + { + return get(InetSocketAddress.createUnresolved(host, port), transportFactory); + } + + public TTransport borrowObject(String host, int port) + { + return get(InetSocketAddress.createUnresolved(host, port)); + } + + private static void closeQuietly(Closeable closeable) + { + try { + closeable.close(); + } + catch (IOException e) { + // ignored + } + } + + public void returnObject(TSocket socket) + { + SocketAddress remote = socket.getSocket().getRemoteSocketAddress() + if (remote == null) + { + return closeQuietly(socket); + } + ObjectPool pool = pools.get(remote); + if (pool == null) + { + return closeQuietly(socket); + } + pool.returnObject(socket); + } + + public void returnObject(TTransport transport) + { + return closeQuietly(transport); + } +} From 46031f8e841341e609813b7ddb97f810e54f5fbf Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 8 Dec 2016 18:19:21 -0800 Subject: [PATCH 117/331] Added PooledTTransportFactory --- .../PooledHiveMetastoreClientFactory.java | 2 +- .../hive/util/PooledTTransportFactory.java | 174 ++++++++++++++++-- .../twitter/hive/util/TTransportPool.java | 2 +- 3 files changed, 164 insertions(+), 14 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java index cea665466ff1..5823d257017f 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java @@ -56,7 +56,7 @@ public HiveMetastoreClient create(String host, int port) try { TTransport transport= transportPool.borrowObject(host, port); if (transport == null) { - transport = transportPool.borrowObject(host, port, new PooledTTransportFactory(host, port, socksProxy, timeoutMillis, metastoreAuthentication)); + transport = transportPool.borrowObject(host, port, new PooledTTransportFactory(transportPool, host, port, socksProxy, timeoutMillis, metastoreAuthentication)); } } catch (Exception e) { diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java index 5e04ed9f64a6..9cda24470d1d 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java @@ -14,25 +14,175 @@ package com.facebook.presto.twitter.hive.util; import com.facebook.presto.hive.authentication.HiveMetastoreAuthentication; - +import com.google.common.net.HostAndPort; +import io.airlift.units.Duration; import javax.annotation.Nullable; - -import java.io.IOException; -import java.net.InetSocketAddress -import java.net.SocketAddress; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - -import org.apache.commons.pool2.ObjectPool; -import org.apache.commons.pool2.impl.GenericObjectPool; +import org.apache.commons.pool2.PooledObject; +import org.apache.commons.pool2.BasePooledObjectFactory; +import org.apache.commons.pool2.impl.DefaultPooledObject; import org.apache.thrift.transport.TSocket; import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; /** * Utility class to handle creating and caching the UserGroupInformation object. */ public class PooledTTransportFactory + extends BasePooledObjectFactory { - + private final TTransportPool pool; + private final String host; + private final int port; + private final HostAndPort socksProxy; + private final Duration timeoutMillis; + private final HiveMetastoreAuthentication metastoreAuthentication; + + public PooledTTransportFactory(TTransportPool pool, String host, int port, @Nullable HostAndPort socksProxy, Duration timeoutMillis, HiveMetastoreAuthentication metastoreAuthentication) + { + this.pool = requireNonNull(pool, "pool is null"); + this.host = requireNonNull(host, "host is null"); + this.port = port; + this.socksProxy = socksProxy; + this.metastoreAuthentication = requireNonNull(metastoreAuthentication, "metastoreAuthentication is null"); + } + + @Override + public TTransport create() + { + return new PooledTTransport(Transport.create(host, port, socksProxy, timeoutMillis, metastoreAuthentication), pool); + } + + @Override + public void destroyObject(PooledObject pooledTransport) + { + try { + ((PooledTTransport) pooledTransport.getObject()).getTTransport().close(); + } + catch (TTransportException e) { + // ignored + } + pooledTransport.invalidate(); + } + + @Override + public PooledObject wrap(TTransport transport) + throws TTransportException + { + return new DefaultPooledObject(transport); + } + + @Override + public void passivateObject(PooledObject pooledTransport) + { + try { + pooledTransport.getObject().flush(); + } + catch (TTransportException e) { + destroyObject(pooledTransport); + } + } + + private static class PooledTTransport + extends TTransport + { + private final TTransportPool pool; + private final TTransport transport; + + public PooledTTransport(TTransport transport, TTransportPool, pool) + { + this.transport = transport; + this.pool = pool; + } + + @Override + public void close() + { + pool.returnObject((TSocket) transport); + } + + @Override + public TTransport.getTTransport() + { + return transport; + } + + @Override + public boolean isOpen() + { + return transport.isOpen(); + } + + @Override + public boolean peek() + { + return transport.peek(); + } + + @Override + public byte[] getBuffer() + { + return transport.getBuffer(); + } + + @Override + public int getBufferPosition() + { + return transport.getBufferPosition(); + } + + @Override + public int getBytesRemainingInBuffer() + { + return transport.getBytesRemainingInBuffer(); + } + + @Override + public void consumeBuffer(int len) + { + transport.consumeBuffer(len); + } + + @Override + public void open() + throws TTransportException + { + transport.open(); + } + + @Override + public int readAll(byte[] bytes, int off, int len) + throws TTransportException + { + return transport.readAll(bytes, off, len); + } + + @Override + public int read(byte[] bytes, int off, int len) + throws TTransportException + { + return transport.read(bytes, off, len); + } + + @Override + public void write(byte[] bytes) + throws TTransportException + { + transport.write(bytes); + } + + @Override + public void write(byte[] bytes, int off, int len) + throws TTransportException + { + transport.write(bytes, off, len); + } + + @Override + public void flush() + throws TTransportException + { + transport.flush(); + } + } + } diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java index b148404a7ff6..ad189d95e7a8 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java @@ -18,7 +18,7 @@ import javax.annotation.Nullable; import java.io.IOException; -import java.net.InetSocketAddress +import java.net.InetSocketAddress; import java.net.SocketAddress; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; From 94991f204aff8efb8d84e5fa61fa47f0dfdfeb5d Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 8 Dec 2016 23:05:24 -0800 Subject: [PATCH 118/331] Correct errors --- pom.xml | 6 ++ presto-hive/pom.xml | 5 ++ .../presto/hive/HiveClientModule.java | 2 + .../PooledHiveMetastoreClientFactory.java | 7 ++- .../hive/ZookeeperServersetHiveCluster.java | 5 +- .../hive/util/PooledTTransportFactory.java | 44 ++++++------- .../twitter/hive/util/TTransportPool.java | 62 ++++++++----------- .../presto/hive/TestingHiveCluster.java | 3 +- 8 files changed, 71 insertions(+), 63 deletions(-) diff --git a/pom.xml b/pom.xml index 77381e4f9179..6458b946f779 100644 --- a/pom.xml +++ b/pom.xml @@ -644,6 +644,12 @@ 3.2 + + org.apache.commons + commons-pool2 + 2.4.2 + + commons-codec commons-codec diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 94c8f9f305e7..7eef88e3c083 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -96,6 +96,11 @@ libthrift + + org.apache.commons + commons-pool2 + + io.airlift aircompressor 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 25de3499bccc..7a187f48015c 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 @@ -31,6 +31,7 @@ import com.facebook.presto.spi.connector.ConnectorPageSourceProvider; import com.facebook.presto.spi.connector.ConnectorSplitManager; import com.facebook.presto.spi.type.TypeManager; +import com.facebook.presto.twitter.hive.PooledHiveMetastoreClientFactory; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; @@ -99,6 +100,7 @@ public void configure(Binder binder) newExporter(binder).export(NamenodeStats.class).as(generatedNameOf(NamenodeStats.class)); binder.bind(HiveMetastoreClientFactory.class).in(Scopes.SINGLETON); + binder.bind(PooledHiveMetastoreClientFactory.class).in(Scopes.SINGLETON); binder.bind(NodeManager.class).toInstance(nodeManager); binder.bind(TypeManager.class).toInstance(typeManager); diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java index 5823d257017f..41fad29fa09b 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java @@ -35,6 +35,7 @@ public class PooledHiveMetastoreClientFactory private final HostAndPort socksProxy; private final int timeoutMillis; private final HiveMetastoreAuthentication metastoreAuthentication; + private final TTransportPool transportPool; public PooledHiveMetastoreClientFactory(@Nullable HostAndPort socksProxy, Duration timeout, HiveMetastoreAuthentication metastoreAuthentication) { @@ -54,14 +55,14 @@ public HiveMetastoreClient create(String host, int port) throws TTransportException { try { - TTransport transport= transportPool.borrowObject(host, port); + TTransport transport = transportPool.borrowObject(host, port); if (transport == null) { transport = transportPool.borrowObject(host, port, new PooledTTransportFactory(transportPool, host, port, socksProxy, timeoutMillis, metastoreAuthentication)); } + return new ThriftHiveMetastoreClient(transport); } catch (Exception e) { - throw new TTransportException(e.getType(), String.format("%s: %s", host, e.getMessage()), e.getCause()) + throw new TTransportException(String.format("%s: %s", host, e.getMessage()), e.getCause()); } - return new ThriftHiveMetastoreClient(transport); } } diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java index 379352d4100e..c4f803311b80 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java @@ -14,7 +14,6 @@ package com.facebook.presto.twitter.hive; import com.facebook.presto.hive.HiveCluster; -import com.facebook.presto.hive.HiveMetastoreClientFactory; import com.facebook.presto.hive.metastore.HiveMetastoreClient; import com.google.common.net.HostAndPort; import io.airlift.log.Logger; @@ -31,11 +30,11 @@ public class ZookeeperServersetHiveCluster implements HiveCluster { private static final Logger log = Logger.get(ZookeeperServersetHiveCluster.class); - private final HiveMetastoreClientFactory clientFactory; + private final PooledHiveMetastoreClientFactory clientFactory; private ZookeeperMetastoreMonitor zkMetastoreMonitor; @Inject - public ZookeeperServersetHiveCluster(ZookeeperServersetMetastoreConfig config, HiveMetastoreClientFactory clientFactory) + public ZookeeperServersetHiveCluster(ZookeeperServersetMetastoreConfig config, PooledHiveMetastoreClientFactory clientFactory) throws Exception { String zkServerHostAndPort = requireNonNull(config.getZookeeperServerHostAndPort(), "zkServerHostAndPort is null"); diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java index 9cda24470d1d..ed465d89503f 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java @@ -14,16 +14,19 @@ package com.facebook.presto.twitter.hive.util; import com.facebook.presto.hive.authentication.HiveMetastoreAuthentication; +import com.facebook.presto.hive.thrift.Transport; import com.google.common.net.HostAndPort; -import io.airlift.units.Duration; -import javax.annotation.Nullable; -import org.apache.commons.pool2.PooledObject; import org.apache.commons.pool2.BasePooledObjectFactory; +import org.apache.commons.pool2.PooledObject; import org.apache.commons.pool2.impl.DefaultPooledObject; import org.apache.thrift.transport.TSocket; import org.apache.thrift.transport.TTransport; import org.apache.thrift.transport.TTransportException; +import javax.annotation.Nullable; + +import static java.util.Objects.requireNonNull; + /** * Utility class to handle creating and caching the UserGroupInformation object. */ @@ -34,51 +37,52 @@ public class PooledTTransportFactory private final String host; private final int port; private final HostAndPort socksProxy; - private final Duration timeoutMillis; + private final int timeoutMillis; private final HiveMetastoreAuthentication metastoreAuthentication; - public PooledTTransportFactory(TTransportPool pool, String host, int port, @Nullable HostAndPort socksProxy, Duration timeoutMillis, HiveMetastoreAuthentication metastoreAuthentication) + public PooledTTransportFactory(TTransportPool pool, String host, int port, @Nullable HostAndPort socksProxy, int timeoutMillis, HiveMetastoreAuthentication metastoreAuthentication) { this.pool = requireNonNull(pool, "pool is null"); this.host = requireNonNull(host, "host is null"); this.port = port; this.socksProxy = socksProxy; + this.timeoutMillis = timeoutMillis; this.metastoreAuthentication = requireNonNull(metastoreAuthentication, "metastoreAuthentication is null"); } @Override public TTransport create() + throws Exception { return new PooledTTransport(Transport.create(host, port, socksProxy, timeoutMillis, metastoreAuthentication), pool); } @Override - public void destroyObject(PooledObject pooledTransport) + public void destroyObject(PooledObject pooledObject) { try { - ((PooledTTransport) pooledTransport.getObject()).getTTransport().close(); + ((PooledTTransport) pooledObject.getObject()).getTTransport().close(); } - catch (TTransportException e) { - // ignored + catch (ClassCastException e) { + // ignore } - pooledTransport.invalidate(); + pooledObject.invalidate(); } @Override public PooledObject wrap(TTransport transport) - throws TTransportException { return new DefaultPooledObject(transport); } @Override - public void passivateObject(PooledObject pooledTransport) + public void passivateObject(PooledObject pooledObject) { try { - pooledTransport.getObject().flush(); + pooledObject.getObject().flush(); } catch (TTransportException e) { - destroyObject(pooledTransport); + destroyObject(pooledObject); } } @@ -88,22 +92,21 @@ private static class PooledTTransport private final TTransportPool pool; private final TTransport transport; - public PooledTTransport(TTransport transport, TTransportPool, pool) + public PooledTTransport(TTransport transport, TTransportPool pool) { this.transport = transport; this.pool = pool; } - @Override - public void close() + public TTransport getTTransport() { - pool.returnObject((TSocket) transport); + return transport; } @Override - public TTransport.getTTransport() + public void close() { - return transport; + pool.returnObject((TSocket) transport); } @Override @@ -184,5 +187,4 @@ public void flush() transport.flush(); } } - } diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java index ad189d95e7a8..cf1b4a432592 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java @@ -13,22 +13,17 @@ */ package com.facebook.presto.twitter.hive.util; -import com.facebook.presto.hive.authentication.HiveMetastoreAuthentication; - -import javax.annotation.Nullable; +import org.apache.commons.pool2.ObjectPool; +import org.apache.commons.pool2.PooledObjectFactory; +import org.apache.commons.pool2.impl.GenericObjectPool; +import org.apache.thrift.transport.TSocket; +import org.apache.thrift.transport.TTransport; -import java.io.IOException; import java.net.InetSocketAddress; import java.net.SocketAddress; -import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import org.apache.commons.pool2.ObjectPool; -import org.apache.commons.pool2.impl.GenericObjectPool; -import org.apache.thrift.transport.TSocket; -import org.apache.thrift.transport.TTransport; - /** * Utility class to handle creating and caching the UserGroupInformation object. */ @@ -44,10 +39,10 @@ private void add(SocketAddress remote, PooledObjectFactory transportFactory) } protected TTransport get(SocketAddress remote, PooledObjectFactory transportFactory) + throws Exception { - ObjectPool pool = pools.get(remote) - if (pool == null) - { + ObjectPool pool = pools.get(remote); + if (pool == null) { add(remote, transportFactory); pool = pools.get(remote); } @@ -56,10 +51,10 @@ protected TTransport get(SocketAddress remote, PooledObjectFactory transportFact } protected TTransport get(SocketAddress remote) + throws Exception { - ObjectPool pool = pools.get(remote) - if (pool == null) - { + ObjectPool pool = pools.get(remote); + if (pool == null) { return null; } @@ -67,42 +62,39 @@ protected TTransport get(SocketAddress remote) } public TTransport borrowObject(String host, int port, PooledObjectFactory transportFactory) + throws Exception { return get(InetSocketAddress.createUnresolved(host, port), transportFactory); } public TTransport borrowObject(String host, int port) + throws Exception { return get(InetSocketAddress.createUnresolved(host, port)); } - private static void closeQuietly(Closeable closeable) - { - try { - closeable.close(); - } - catch (IOException e) { - // ignored - } - } - public void returnObject(TSocket socket) { - SocketAddress remote = socket.getSocket().getRemoteSocketAddress() - if (remote == null) - { - return closeQuietly(socket); + SocketAddress remote = socket.getSocket().getRemoteSocketAddress(); + if (remote == null) { + socket.close(); + return; } ObjectPool pool = pools.get(remote); - if (pool == null) - { - return closeQuietly(socket); + if (pool == null) { + socket.close(); + return; + } + try { + pool.returnObject(socket); + } + catch (Exception e) { + // ignored } - pool.returnObject(socket); } public void returnObject(TTransport transport) { - return closeQuietly(transport); + transport.close(); } } diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestingHiveCluster.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestingHiveCluster.java index c35f682a03c8..6b8ac2370e90 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestingHiveCluster.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestingHiveCluster.java @@ -15,6 +15,7 @@ import com.facebook.presto.hive.authentication.NoHiveMetastoreAuthentication; import com.facebook.presto.hive.metastore.HiveMetastoreClient; +import com.facebook.presto.twitter.hive.PooledHiveMetastoreClientFactory; import com.google.common.base.Throwables; import org.apache.thrift.transport.TTransportException; @@ -40,7 +41,7 @@ public TestingHiveCluster(HiveClientConfig config, String host, int port) public HiveMetastoreClient createMetastoreClient() { try { - return new HiveMetastoreClientFactory(config, new NoHiveMetastoreAuthentication()).create(host, port); + return new PooledHiveMetastoreClientFactory(config, new NoHiveMetastoreAuthentication()).create(host, port); } catch (TTransportException e) { throw Throwables.propagate(e); From 0f1a82669a640d9b22bd58fb56b15c06515f91df Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 9 Dec 2016 02:17:25 -0800 Subject: [PATCH 119/331] Remove Transport in PooledObjectFactory --- .../hive/util/PooledTTransportFactory.java | 41 +++++++++++++++++-- 1 file changed, 38 insertions(+), 3 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java index ed465d89503f..2a87a19e7f2d 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java @@ -14,7 +14,6 @@ package com.facebook.presto.twitter.hive.util; import com.facebook.presto.hive.authentication.HiveMetastoreAuthentication; -import com.facebook.presto.hive.thrift.Transport; import com.google.common.net.HostAndPort; import org.apache.commons.pool2.BasePooledObjectFactory; import org.apache.commons.pool2.PooledObject; @@ -25,6 +24,13 @@ import javax.annotation.Nullable; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.Proxy; +import java.net.Socket; +import java.net.SocketAddress; +import java.net.SocketException; + import static java.util.Objects.requireNonNull; /** @@ -54,7 +60,36 @@ public PooledTTransportFactory(TTransportPool pool, String host, int port, @Null public TTransport create() throws Exception { - return new PooledTTransport(Transport.create(host, port, socksProxy, timeoutMillis, metastoreAuthentication), pool); + TTransport transport; + if (socksProxy == null) { + transport = new TSocket(host, port, timeoutMillis); + } + else { + SocketAddress address = InetSocketAddress.createUnresolved(socksProxy.getHostText(), socksProxy.getPort()); + Socket socket = new Socket(new Proxy(Proxy.Type.SOCKS, address)); + try { + socket.connect(InetSocketAddress.createUnresolved(host, port), timeoutMillis); + socket.setSoTimeout(timeoutMillis); + transport = new TSocket(socket); + } + catch (SocketException e) { + if (socket.isConnected()) { + try { + socket.close(); + } + catch (IOException ioEexception) { + // ignored + } + } + throw e; + } + } + TTransport authenticatedTransport = metastoreAuthentication.authenticate(transport, host); + if (!authenticatedTransport.isOpen()) { + authenticatedTransport.open(); + } + + return new PooledTTransport(authenticatedTransport, pool); } @Override @@ -106,7 +141,7 @@ public TTransport getTTransport() @Override public void close() { - pool.returnObject((TSocket) transport); + pool.returnObject(transport); } @Override From 65e2bea910311f4921dd830be29250faf53cb6d9 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 9 Dec 2016 11:20:00 -0800 Subject: [PATCH 120/331] Added logs --- .../hive/PooledHiveMetastoreClientFactory.java | 3 +++ .../twitter/hive/util/PooledTTransportFactory.java | 10 ++++++++++ .../presto/twitter/hive/util/TTransportPool.java | 13 +++++++++++-- 3 files changed, 24 insertions(+), 2 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java index 41fad29fa09b..87efde6a62fb 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java @@ -21,6 +21,7 @@ import com.facebook.presto.twitter.hive.util.TTransportPool; import com.google.common.net.HostAndPort; import com.google.common.primitives.Ints; +import io.airlift.log.Logger; import io.airlift.units.Duration; import org.apache.thrift.transport.TTransport; import org.apache.thrift.transport.TTransportException; @@ -32,6 +33,7 @@ public class PooledHiveMetastoreClientFactory { + private static final Logger log = Logger.get(PooledHiveMetastoreClientFactory.class); private final HostAndPort socksProxy; private final int timeoutMillis; private final HiveMetastoreAuthentication metastoreAuthentication; @@ -59,6 +61,7 @@ public HiveMetastoreClient create(String host, int port) if (transport == null) { transport = transportPool.borrowObject(host, port, new PooledTTransportFactory(transportPool, host, port, socksProxy, timeoutMillis, metastoreAuthentication)); } + log.debug("borrowed a transport for host: %s", host); return new ThriftHiveMetastoreClient(transport); } catch (Exception e) { diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java index 2a87a19e7f2d..ecd0c9b7b176 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java @@ -15,6 +15,7 @@ import com.facebook.presto.hive.authentication.HiveMetastoreAuthentication; import com.google.common.net.HostAndPort; +import io.airlift.log.Logger; import org.apache.commons.pool2.BasePooledObjectFactory; import org.apache.commons.pool2.PooledObject; import org.apache.commons.pool2.impl.DefaultPooledObject; @@ -39,6 +40,7 @@ public class PooledTTransportFactory extends BasePooledObjectFactory { + private static final Logger log = Logger.get(PooledTTransportFactory.class); private final TTransportPool pool; private final String host; private final int port; @@ -60,6 +62,7 @@ public PooledTTransportFactory(TTransportPool pool, String host, int port, @Null public TTransport create() throws Exception { + log.debug("creating a transport to: %s", host); TTransport transport; if (socksProxy == null) { transport = new TSocket(host, port, timeoutMillis); @@ -89,16 +92,19 @@ public TTransport create() authenticatedTransport.open(); } + log.debug("created a transport to: %s", host); return new PooledTTransport(authenticatedTransport, pool); } @Override public void destroyObject(PooledObject pooledObject) { + log.debug("destroy a transport to: %s", host); try { ((PooledTTransport) pooledObject.getObject()).getTTransport().close(); } catch (ClassCastException e) { + log.debug("cannot cast to PooledTTransport"); // ignore } pooledObject.invalidate(); @@ -107,16 +113,19 @@ public void destroyObject(PooledObject pooledObject) @Override public PooledObject wrap(TTransport transport) { + log.debug("wrapping a transport to %s", host); return new DefaultPooledObject(transport); } @Override public void passivateObject(PooledObject pooledObject) { + log.debug("passivate a transport to %s", host); try { pooledObject.getObject().flush(); } catch (TTransportException e) { + log.debug("Failed to flush transport, destroy it"); destroyObject(pooledObject); } } @@ -141,6 +150,7 @@ public TTransport getTTransport() @Override public void close() { + log.debug("attempt to close a PooledTTransport, returning it to pool."); pool.returnObject(transport); } diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java index cf1b4a432592..001f7fe03bbc 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.twitter.hive.util; +import io.airlift.log.Logger; import org.apache.commons.pool2.ObjectPool; import org.apache.commons.pool2.PooledObjectFactory; import org.apache.commons.pool2.impl.GenericObjectPool; @@ -29,12 +30,14 @@ */ public class TTransportPool { + private static final Logger log = Logger.get(TTransportPool.class); private final ConcurrentMap> pools = new ConcurrentHashMap(); public TTransportPool(){} private void add(SocketAddress remote, PooledObjectFactory transportFactory) { + log.debug("Added new pool for destination: %s", remote); pools.put(remote, new GenericObjectPool(transportFactory)); } @@ -46,7 +49,7 @@ protected TTransport get(SocketAddress remote, PooledObjectFactory transportFact add(remote, transportFactory); pool = pools.get(remote); } - + log.debug("Fetched transport pool for : %s", remote); return pool.borrowObject(); } @@ -55,9 +58,11 @@ protected TTransport get(SocketAddress remote) { ObjectPool pool = pools.get(remote); if (pool == null) { + log.debug("Doesn't have transport for : %s", remote); return null; } + log.debug("Fetched transport pool for : %s", remote); return pool.borrowObject(); } @@ -76,25 +81,29 @@ public TTransport borrowObject(String host, int port) public void returnObject(TSocket socket) { SocketAddress remote = socket.getSocket().getRemoteSocketAddress(); + log.debug("Return a socket to: %s", remote); if (remote == null) { socket.close(); + log.debug("Remote is null"); return; } ObjectPool pool = pools.get(remote); if (pool == null) { socket.close(); + log.debug("Cannot find pool"); return; } try { pool.returnObject(socket); } catch (Exception e) { - // ignored + log.debug("Got an error when return to pool: %s", e.getMessage()); } } public void returnObject(TTransport transport) { + log.debug("Return a transport, close directly"); transport.close(); } } From 090cd4a253d0fc30735b8585d5d00419a4bf29f9 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 9 Dec 2016 11:55:29 -0800 Subject: [PATCH 121/331] Try to cast class before return --- .../presto/twitter/hive/util/PooledTTransportFactory.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java index ecd0c9b7b176..08252220a84a 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java @@ -151,7 +151,12 @@ public TTransport getTTransport() public void close() { log.debug("attempt to close a PooledTTransport, returning it to pool."); - pool.returnObject(transport); + try { + pool.returnObject((TSocket) transport); + } + catch (ClassCastException e) { + pool.returnObject(transport); + } } @Override From 30b6d592b77bfabeba00ca6e18f7cece1b5056ec Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 9 Dec 2016 15:18:15 -0800 Subject: [PATCH 122/331] Changed mapping key as String --- .../hive/util/PooledTTransportFactory.java | 8 +++--- .../twitter/hive/util/TTransportPool.java | 25 +++++++++++-------- 2 files changed, 19 insertions(+), 14 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java index 08252220a84a..3f91cca813c8 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java @@ -93,7 +93,7 @@ public TTransport create() } log.debug("created a transport to: %s", host); - return new PooledTTransport(authenticatedTransport, pool); + return new PooledTTransport(authenticatedTransport, pool, HostAndPort.fromParts(host, port).toString()); } @Override @@ -133,13 +133,15 @@ public void passivateObject(PooledObject pooledObject) private static class PooledTTransport extends TTransport { + private final String remote; private final TTransportPool pool; private final TTransport transport; - public PooledTTransport(TTransport transport, TTransportPool pool) + public PooledTTransport(TTransport transport, TTransportPool pool, String remote) { this.transport = transport; this.pool = pool; + this.remote = remote; } public TTransport getTTransport() @@ -152,7 +154,7 @@ public void close() { log.debug("attempt to close a PooledTTransport, returning it to pool."); try { - pool.returnObject((TSocket) transport); + pool.returnObject(remote, (TSocket) transport); } catch (ClassCastException e) { pool.returnObject(transport); diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java index 001f7fe03bbc..6b76b591eb3c 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.twitter.hive.util; +import com.google.common.net.HostAndPort; import io.airlift.log.Logger; import org.apache.commons.pool2.ObjectPool; import org.apache.commons.pool2.PooledObjectFactory; @@ -20,8 +21,6 @@ import org.apache.thrift.transport.TSocket; import org.apache.thrift.transport.TTransport; -import java.net.InetSocketAddress; -import java.net.SocketAddress; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -31,17 +30,17 @@ public class TTransportPool { private static final Logger log = Logger.get(TTransportPool.class); - private final ConcurrentMap> pools = new ConcurrentHashMap(); + private final ConcurrentMap> pools = new ConcurrentHashMap(); public TTransportPool(){} - private void add(SocketAddress remote, PooledObjectFactory transportFactory) + private void add(String remote, PooledObjectFactory transportFactory) { log.debug("Added new pool for destination: %s", remote); pools.put(remote, new GenericObjectPool(transportFactory)); } - protected TTransport get(SocketAddress remote, PooledObjectFactory transportFactory) + protected TTransport get(String remote, PooledObjectFactory transportFactory) throws Exception { ObjectPool pool = pools.get(remote); @@ -53,7 +52,7 @@ protected TTransport get(SocketAddress remote, PooledObjectFactory transportFact return pool.borrowObject(); } - protected TTransport get(SocketAddress remote) + protected TTransport get(String remote) throws Exception { ObjectPool pool = pools.get(remote); @@ -69,18 +68,17 @@ protected TTransport get(SocketAddress remote) public TTransport borrowObject(String host, int port, PooledObjectFactory transportFactory) throws Exception { - return get(InetSocketAddress.createUnresolved(host, port), transportFactory); + return get(HostAndPort.fromParts(host, port).toString(), transportFactory); } public TTransport borrowObject(String host, int port) throws Exception { - return get(InetSocketAddress.createUnresolved(host, port)); + return get(HostAndPort.fromParts(host, port).toString()); } - public void returnObject(TSocket socket) + public void returnObject(String remote, TSocket socket) { - SocketAddress remote = socket.getSocket().getRemoteSocketAddress(); log.debug("Return a socket to: %s", remote); if (remote == null) { socket.close(); @@ -101,9 +99,14 @@ public void returnObject(TSocket socket) } } + public void returnObject(String remote, TTransport transport) + { + returnObject(transport); + } + public void returnObject(TTransport transport) { - log.debug("Return a transport, close directly"); + log.debug("Return a ttransport, close directly"); transport.close(); } } From 646106a717cce9e61da3459b279b76eb2f00956e Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 9 Dec 2016 16:28:08 -0800 Subject: [PATCH 123/331] Removed logging --- .../PooledHiveMetastoreClientFactory.java | 3 --- .../hive/util/PooledTTransportFactory.java | 18 +++---------- .../twitter/hive/util/TTransportPool.java | 25 +++---------------- 3 files changed, 8 insertions(+), 38 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java index 87efde6a62fb..41fad29fa09b 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java @@ -21,7 +21,6 @@ import com.facebook.presto.twitter.hive.util.TTransportPool; import com.google.common.net.HostAndPort; import com.google.common.primitives.Ints; -import io.airlift.log.Logger; import io.airlift.units.Duration; import org.apache.thrift.transport.TTransport; import org.apache.thrift.transport.TTransportException; @@ -33,7 +32,6 @@ public class PooledHiveMetastoreClientFactory { - private static final Logger log = Logger.get(PooledHiveMetastoreClientFactory.class); private final HostAndPort socksProxy; private final int timeoutMillis; private final HiveMetastoreAuthentication metastoreAuthentication; @@ -61,7 +59,6 @@ public HiveMetastoreClient create(String host, int port) if (transport == null) { transport = transportPool.borrowObject(host, port, new PooledTTransportFactory(transportPool, host, port, socksProxy, timeoutMillis, metastoreAuthentication)); } - log.debug("borrowed a transport for host: %s", host); return new ThriftHiveMetastoreClient(transport); } catch (Exception e) { diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java index 3f91cca813c8..17b26479583e 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java @@ -15,7 +15,6 @@ import com.facebook.presto.hive.authentication.HiveMetastoreAuthentication; import com.google.common.net.HostAndPort; -import io.airlift.log.Logger; import org.apache.commons.pool2.BasePooledObjectFactory; import org.apache.commons.pool2.PooledObject; import org.apache.commons.pool2.impl.DefaultPooledObject; @@ -40,7 +39,6 @@ public class PooledTTransportFactory extends BasePooledObjectFactory { - private static final Logger log = Logger.get(PooledTTransportFactory.class); private final TTransportPool pool; private final String host; private final int port; @@ -62,7 +60,6 @@ public PooledTTransportFactory(TTransportPool pool, String host, int port, @Null public TTransport create() throws Exception { - log.debug("creating a transport to: %s", host); TTransport transport; if (socksProxy == null) { transport = new TSocket(host, port, timeoutMillis); @@ -80,7 +77,7 @@ public TTransport create() try { socket.close(); } - catch (IOException ioEexception) { + catch (IOException ioException) { // ignored } } @@ -92,19 +89,16 @@ public TTransport create() authenticatedTransport.open(); } - log.debug("created a transport to: %s", host); return new PooledTTransport(authenticatedTransport, pool, HostAndPort.fromParts(host, port).toString()); } @Override public void destroyObject(PooledObject pooledObject) { - log.debug("destroy a transport to: %s", host); try { ((PooledTTransport) pooledObject.getObject()).getTTransport().close(); } catch (ClassCastException e) { - log.debug("cannot cast to PooledTTransport"); // ignore } pooledObject.invalidate(); @@ -113,19 +107,16 @@ public void destroyObject(PooledObject pooledObject) @Override public PooledObject wrap(TTransport transport) { - log.debug("wrapping a transport to %s", host); return new DefaultPooledObject(transport); } @Override public void passivateObject(PooledObject pooledObject) { - log.debug("passivate a transport to %s", host); try { pooledObject.getObject().flush(); } catch (TTransportException e) { - log.debug("Failed to flush transport, destroy it"); destroyObject(pooledObject); } } @@ -152,12 +143,11 @@ public TTransport getTTransport() @Override public void close() { - log.debug("attempt to close a PooledTTransport, returning it to pool."); try { - pool.returnObject(remote, (TSocket) transport); + pool.returnObject(remote, this, transport); } - catch (ClassCastException e) { - pool.returnObject(transport); + catch (Exception e) { + transport.close(); } } diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java index 6b76b591eb3c..20effdf46967 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java @@ -14,11 +14,9 @@ package com.facebook.presto.twitter.hive.util; import com.google.common.net.HostAndPort; -import io.airlift.log.Logger; import org.apache.commons.pool2.ObjectPool; import org.apache.commons.pool2.PooledObjectFactory; import org.apache.commons.pool2.impl.GenericObjectPool; -import org.apache.thrift.transport.TSocket; import org.apache.thrift.transport.TTransport; import java.util.concurrent.ConcurrentHashMap; @@ -29,14 +27,12 @@ */ public class TTransportPool { - private static final Logger log = Logger.get(TTransportPool.class); private final ConcurrentMap> pools = new ConcurrentHashMap(); public TTransportPool(){} private void add(String remote, PooledObjectFactory transportFactory) { - log.debug("Added new pool for destination: %s", remote); pools.put(remote, new GenericObjectPool(transportFactory)); } @@ -48,7 +44,6 @@ protected TTransport get(String remote, PooledObjectFactory transportFactory) add(remote, transportFactory); pool = pools.get(remote); } - log.debug("Fetched transport pool for : %s", remote); return pool.borrowObject(); } @@ -57,11 +52,9 @@ protected TTransport get(String remote) { ObjectPool pool = pools.get(remote); if (pool == null) { - log.debug("Doesn't have transport for : %s", remote); return null; } - log.debug("Fetched transport pool for : %s", remote); return pool.borrowObject(); } @@ -77,36 +70,26 @@ public TTransport borrowObject(String host, int port) return get(HostAndPort.fromParts(host, port).toString()); } - public void returnObject(String remote, TSocket socket) + public void returnObject(String remote, TTransport pooledTransport, TTransport transport) { - log.debug("Return a socket to: %s", remote); if (remote == null) { - socket.close(); - log.debug("Remote is null"); + transport.close(); return; } ObjectPool pool = pools.get(remote); if (pool == null) { - socket.close(); - log.debug("Cannot find pool"); + transport.close(); return; } try { - pool.returnObject(socket); + pool.returnObject(pooledTransport); } catch (Exception e) { - log.debug("Got an error when return to pool: %s", e.getMessage()); } } - public void returnObject(String remote, TTransport transport) - { - returnObject(transport); - } - public void returnObject(TTransport transport) { - log.debug("Return a ttransport, close directly"); transport.close(); } } From eca3b40f1a7cef02c8c3e23f7dcf2924730feecf Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Mon, 12 Dec 2016 12:33:43 -0800 Subject: [PATCH 124/331] Added validation, added log to print mean wait time for each pool --- .../hive/util/PooledTTransportFactory.java | 9 ++++--- .../twitter/hive/util/TTransportPool.java | 27 ++++++++++++++----- 2 files changed, 27 insertions(+), 9 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java index 17b26479583e..b7896754351a 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java @@ -33,9 +33,6 @@ import static java.util.Objects.requireNonNull; -/** - * Utility class to handle creating and caching the UserGroupInformation object. - */ public class PooledTTransportFactory extends BasePooledObjectFactory { @@ -56,6 +53,12 @@ public PooledTTransportFactory(TTransportPool pool, String host, int port, @Null this.metastoreAuthentication = requireNonNull(metastoreAuthentication, "metastoreAuthentication is null"); } + @Override + public boolean validateObject(PooledObject pooledObject) + { + return pooledObject.getObject().isOpen(); + } + @Override public TTransport create() throws Exception diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java index 20effdf46967..7836f92b5c09 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java @@ -14,26 +14,41 @@ package com.facebook.presto.twitter.hive.util; import com.google.common.net.HostAndPort; +import io.airlift.log.Logger; import org.apache.commons.pool2.ObjectPool; import org.apache.commons.pool2.PooledObjectFactory; import org.apache.commons.pool2.impl.GenericObjectPool; +import org.apache.commons.pool2.impl.GenericObjectPoolConfig; import org.apache.thrift.transport.TTransport; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -/** - * Utility class to handle creating and caching the UserGroupInformation object. - */ public class TTransportPool { + public static final Logger log = Logger.get(TTransportPool.class); + private static final int MAX_IDLE = 8; + private static final int MIN_IDLE = 0; + private static final int MAX_TOTAL = 100; private final ConcurrentMap> pools = new ConcurrentHashMap(); + private GenericObjectPoolConfig poolConfig; - public TTransportPool(){} + public TTransportPool() + { + poolConfig = new GenericObjectPoolConfig(); + poolConfig.setMaxIdle(MAX_IDLE); + poolConfig.setMinIdle(MIN_IDLE); + poolConfig.setMaxTotal(MAX_TOTAL); + } + + public TTransportPool(GenericObjectPoolConfig poolConfig) + { + this.poolConfig = poolConfig; + } private void add(String remote, PooledObjectFactory transportFactory) { - pools.put(remote, new GenericObjectPool(transportFactory)); + pools.put(remote, new GenericObjectPool(transportFactory, poolConfig)); } protected TTransport get(String remote, PooledObjectFactory transportFactory) @@ -54,7 +69,7 @@ protected TTransport get(String remote) if (pool == null) { return null; } - + log.debug("The mean borrow wait time for %s is %d millis", remote, ((GenericObjectPool) pool).getMeanBorrowWaitTimeMillis()); return pool.borrowObject(); } From 9c0e9f2714aa5d0f15dae662f5f23a14d39e7f2a Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Mon, 12 Dec 2016 12:55:28 -0800 Subject: [PATCH 125/331] Added more values for logging --- .../facebook/presto/twitter/hive/util/TTransportPool.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java index 7836f92b5c09..d06635e70de5 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java @@ -29,7 +29,7 @@ public class TTransportPool public static final Logger log = Logger.get(TTransportPool.class); private static final int MAX_IDLE = 8; private static final int MIN_IDLE = 0; - private static final int MAX_TOTAL = 100; + private static final int MAX_TOTAL = 128; private final ConcurrentMap> pools = new ConcurrentHashMap(); private GenericObjectPoolConfig poolConfig; @@ -69,7 +69,11 @@ protected TTransport get(String remote) if (pool == null) { return null; } - log.debug("The mean borrow wait time for %s is %d millis", remote, ((GenericObjectPool) pool).getMeanBorrowWaitTimeMillis()); + log.debug("Pool %s: mean wait time: %d millis, borrowed %d, idle %d.", + remote, + ((GenericObjectPool) pool).getMeanBorrowWaitTimeMillis(), + ((GenericObjectPool) pool).getNumActive(), + ((GenericObjectPool) pool).getNumIdle()); return pool.borrowObject(); } From 6123869f83d488dee22c22ca089dd4f4dc77be4d Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Mon, 12 Dec 2016 16:36:00 -0800 Subject: [PATCH 126/331] Set Eviction Configs --- .../twitter/hive/util/TTransportPool.java | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java index d06635e70de5..7a435dd438cd 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java @@ -14,7 +14,7 @@ package com.facebook.presto.twitter.hive.util; import com.google.common.net.HostAndPort; -import io.airlift.log.Logger; +import io.airlift.units.Duration; import org.apache.commons.pool2.ObjectPool; import org.apache.commons.pool2.PooledObjectFactory; import org.apache.commons.pool2.impl.GenericObjectPool; @@ -23,22 +23,27 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; public class TTransportPool { - public static final Logger log = Logger.get(TTransportPool.class); - private static final int MAX_IDLE = 8; private static final int MIN_IDLE = 0; private static final int MAX_TOTAL = 128; + private static final int NUM_TESTS = 3; + private static final Duration IDLE_TIMEOUT = new Duration(300, TimeUnit.SECONDS); + private static final Duration EVICTION_INTERVEL = new Duration(10, TimeUnit.SECONDS); private final ConcurrentMap> pools = new ConcurrentHashMap(); private GenericObjectPoolConfig poolConfig; public TTransportPool() { poolConfig = new GenericObjectPoolConfig(); - poolConfig.setMaxIdle(MAX_IDLE); + poolConfig.setMaxIdle(MAX_TOTAL); poolConfig.setMinIdle(MIN_IDLE); poolConfig.setMaxTotal(MAX_TOTAL); + poolConfig.setMinEvictableIdleTimeMillis(IDLE_TIMEOUT.toMillis()); + poolConfig.setTimeBetweenEvictionRunsMillis(EVICTION_INTERVEL.toMillis()); + poolConfig.setNumTestsPerEvictionRun(NUM_TESTS); } public TTransportPool(GenericObjectPoolConfig poolConfig) @@ -69,11 +74,6 @@ protected TTransport get(String remote) if (pool == null) { return null; } - log.debug("Pool %s: mean wait time: %d millis, borrowed %d, idle %d.", - remote, - ((GenericObjectPool) pool).getMeanBorrowWaitTimeMillis(), - ((GenericObjectPool) pool).getNumActive(), - ((GenericObjectPool) pool).getNumIdle()); return pool.borrowObject(); } @@ -104,6 +104,7 @@ public void returnObject(String remote, TTransport pooledTransport, TTransport t pool.returnObject(pooledTransport); } catch (Exception e) { + transport.close(); } } From cdc7433184b1139fde6fb18e7a3d37449d4de35b Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 13 Dec 2016 16:11:56 -0800 Subject: [PATCH 127/331] Added support to pass transport pool config --- .../PooledHiveMetastoreClientFactory.java | 17 ++++-- .../ZookeeperServersetMetastoreConfig.java | 54 +++++++++++++++++++ .../twitter/hive/util/TTransportPool.java | 4 +- .../presto/hive/TestingHiveCluster.java | 3 +- 4 files changed, 70 insertions(+), 8 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java index 41fad29fa09b..e411ffa879fe 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java @@ -22,6 +22,7 @@ import com.google.common.net.HostAndPort; import com.google.common.primitives.Ints; import io.airlift.units.Duration; +import org.apache.commons.pool2.impl.GenericObjectPoolConfig; import org.apache.thrift.transport.TTransport; import org.apache.thrift.transport.TTransportException; @@ -37,18 +38,26 @@ public class PooledHiveMetastoreClientFactory private final HiveMetastoreAuthentication metastoreAuthentication; private final TTransportPool transportPool; - public PooledHiveMetastoreClientFactory(@Nullable HostAndPort socksProxy, Duration timeout, HiveMetastoreAuthentication metastoreAuthentication) + public PooledHiveMetastoreClientFactory(@Nullable HostAndPort socksProxy, Duration timeout, HiveMetastoreAuthentication metastoreAuthentication, + int maxTransport, long idleTimeout, long transportEvictInterval, int evictNumTests) { this.socksProxy = socksProxy; this.timeoutMillis = Ints.checkedCast(timeout.toMillis()); this.metastoreAuthentication = requireNonNull(metastoreAuthentication, "metastoreAuthentication is null"); - this.transportPool = new TTransportPool(); + GenericObjectPoolConfig poolConfig = new GenericObjectPoolConfig(); + poolConfig.setMaxIdle(maxTransport); + poolConfig.setMaxTotal(maxTransport); + poolConfig.setMinEvictableIdleTimeMillis(idleTimeout); + poolConfig.setTimeBetweenEvictionRunsMillis(transportEvictInterval); + poolConfig.setNumTestsPerEvictionRun(evictNumTests); + this.transportPool = new TTransportPool(poolConfig); } @Inject - public PooledHiveMetastoreClientFactory(HiveClientConfig config, HiveMetastoreAuthentication metastoreAuthentication) + public PooledHiveMetastoreClientFactory(HiveClientConfig config, ZookeeperServersetMetastoreConfig zkConfig, HiveMetastoreAuthentication metastoreAuthentication) { - this(config.getMetastoreSocksProxy(), config.getMetastoreTimeout(), metastoreAuthentication); + this(config.getMetastoreSocksProxy(), config.getMetastoreTimeout(), metastoreAuthentication, + zkConfig.getMaxTransport(), zkConfig.getTransportIdleTimeout(), zkConfig.getTransportEvictInterval(), zkConfig.getTransportEvictNumTests()); } public HiveMetastoreClient create(String host, int port) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetMetastoreConfig.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetMetastoreConfig.java index 26e36b469d0c..65b424b6c437 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetMetastoreConfig.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetMetastoreConfig.java @@ -25,6 +25,10 @@ public class ZookeeperServersetMetastoreConfig private String zookeeperMetastorePath; private int zookeeperRetrySleepTime = 500; // ms private int zookeeperMaxRetries = 3; + private int maxTransport = 128; + private long transportIdleTimeout = 300_000L; + private long transportEvictInterval = 10_000L; + private int transportEvictNumTests = 3; public String getZookeeperServerHostAndPort() { @@ -79,4 +83,54 @@ public ZookeeperServersetMetastoreConfig setZookeeperMaxRetries(int zookeeperMax this.zookeeperMaxRetries = zookeeperMaxRetries; return this; } + + @Min(1) + public int getMaxTransport() + { + return maxTransport; + } + + @Config("hive.metastore.max-transport-num") + public ZookeeperServersetMetastoreConfig setMaxTransport(int maxTransport) + { + this.maxTransport = maxTransport; + return this; + } + + public long getTransportIdleTimeout() + { + return transportIdleTimeout; + } + + @Config("hive.metastore.transport-idle-timeout") + public ZookeeperServersetMetastoreConfig setTransportIdleTimeout(long transportIdleTimeout) + { + this.transportIdleTimeout = transportIdleTimeout; + return this; + } + + public long getTransportEvictInterval() + { + return transportEvictInterval; + } + + @Config("hive.metastore.transport-eviction-interval") + public ZookeeperServersetMetastoreConfig setTransportEvictInterval(long transportEvictInterval) + { + this.transportEvictInterval = transportEvictInterval; + return this; + } + + @Min(0) + public int getTransportEvictNumTests() + { + return transportEvictNumTests; + } + + @Config("hive.metastore.transport-eviction-num-tests") + public ZookeeperServersetMetastoreConfig setTransportEvictNumTests(int transportEvictNumTests) + { + this.transportEvictNumTests = transportEvictNumTests; + return this; + } } diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java index 7a435dd438cd..cbb5c58333f1 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java @@ -31,7 +31,7 @@ public class TTransportPool private static final int MAX_TOTAL = 128; private static final int NUM_TESTS = 3; private static final Duration IDLE_TIMEOUT = new Duration(300, TimeUnit.SECONDS); - private static final Duration EVICTION_INTERVEL = new Duration(10, TimeUnit.SECONDS); + private static final Duration EVICTION_INTERVAL = new Duration(10, TimeUnit.SECONDS); private final ConcurrentMap> pools = new ConcurrentHashMap(); private GenericObjectPoolConfig poolConfig; @@ -42,7 +42,7 @@ public TTransportPool() poolConfig.setMinIdle(MIN_IDLE); poolConfig.setMaxTotal(MAX_TOTAL); poolConfig.setMinEvictableIdleTimeMillis(IDLE_TIMEOUT.toMillis()); - poolConfig.setTimeBetweenEvictionRunsMillis(EVICTION_INTERVEL.toMillis()); + poolConfig.setTimeBetweenEvictionRunsMillis(EVICTION_INTERVAL.toMillis()); poolConfig.setNumTestsPerEvictionRun(NUM_TESTS); } diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestingHiveCluster.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestingHiveCluster.java index 6b8ac2370e90..c35f682a03c8 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestingHiveCluster.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestingHiveCluster.java @@ -15,7 +15,6 @@ import com.facebook.presto.hive.authentication.NoHiveMetastoreAuthentication; import com.facebook.presto.hive.metastore.HiveMetastoreClient; -import com.facebook.presto.twitter.hive.PooledHiveMetastoreClientFactory; import com.google.common.base.Throwables; import org.apache.thrift.transport.TTransportException; @@ -41,7 +40,7 @@ public TestingHiveCluster(HiveClientConfig config, String host, int port) public HiveMetastoreClient createMetastoreClient() { try { - return new PooledHiveMetastoreClientFactory(config, new NoHiveMetastoreAuthentication()).create(host, port); + return new HiveMetastoreClientFactory(config, new NoHiveMetastoreAuthentication()).create(host, port); } catch (TTransportException e) { throw Throwables.propagate(e); From 604d9d2d65aa5b5358a79789da7ae42d5365ffea Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 13 Dec 2016 17:57:43 -0800 Subject: [PATCH 128/331] Update ZKMConfig test set --- .../TestZookeeperServersetMetastoreConfig.java | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/presto-hive/src/test/java/com/facebook/presto/twitter/hive/TestZookeeperServersetMetastoreConfig.java b/presto-hive/src/test/java/com/facebook/presto/twitter/hive/TestZookeeperServersetMetastoreConfig.java index 6992a752dea1..b839a8d15404 100644 --- a/presto-hive/src/test/java/com/facebook/presto/twitter/hive/TestZookeeperServersetMetastoreConfig.java +++ b/presto-hive/src/test/java/com/facebook/presto/twitter/hive/TestZookeeperServersetMetastoreConfig.java @@ -31,7 +31,11 @@ public void testDefaults() .setZookeeperMaxRetries(3) .setZookeeperRetrySleepTime(500) .setZookeeperMetastorePath(null) - .setZookeeperServerHostAndPort(null)); + .setZookeeperServerHostAndPort(null) + .setMaxTransport(128) + .setTransportIdleTimeout(300_000L) + .setTransportEvictInterval(10_000L) + .setTransportEvictNumTests(3)); } @Test @@ -42,13 +46,21 @@ public void testExplicitPropertyMappingsSingleMetastore() .put("hive.metastore.zookeeper.path", "/zookeeper/path/") .put("hive.metastore.zookeeper.retry.sleeptime", "200") .put("hive.metastore.zookeeper.max.retries", "2") + .put("hive.metastore.max-transport-num", "64") + .put("hive.metastore.transport-idle-timeout", "100000") + .put("hive.metastore.transport-eviction-interval", "1000") + .put("hive.metastore.transport-eviction-num-tests", "10") .build(); ZookeeperServersetMetastoreConfig expected = new ZookeeperServersetMetastoreConfig() .setZookeeperServerHostAndPort("localhost:2181") .setZookeeperMetastorePath("/zookeeper/path/") .setZookeeperRetrySleepTime(200) - .setZookeeperMaxRetries(2); + .setZookeeperMaxRetries(2) + .setMaxTransport(64) + .setTransportIdleTimeout(100_000L) + .setTransportEvictInterval(1_000L) + .setTransportEvictNumTests(10); assertFullMapping(properties, expected); } From 602acb029d7251d79bfff1bf61e0487bf3107b96 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 13 Dec 2016 21:42:17 -0800 Subject: [PATCH 129/331] Enforce thread safety --- .../presto/twitter/hive/util/TTransportPool.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java index cbb5c58333f1..6042a484bfe5 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java @@ -51,20 +51,16 @@ public TTransportPool(GenericObjectPoolConfig poolConfig) this.poolConfig = poolConfig; } - private void add(String remote, PooledObjectFactory transportFactory) + protected synchronized void add(String remote, PooledObjectFactory transportFactory) { - pools.put(remote, new GenericObjectPool(transportFactory, poolConfig)); + pools.putIfAbsent(remote, new GenericObjectPool(transportFactory, poolConfig)); } protected TTransport get(String remote, PooledObjectFactory transportFactory) throws Exception { - ObjectPool pool = pools.get(remote); - if (pool == null) { - add(remote, transportFactory); - pool = pools.get(remote); - } - return pool.borrowObject(); + add(remote, transportFactory); + return get(remote); } protected TTransport get(String remote) From 518c27edf249d60e569a033460649382cf278571 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 14 Dec 2016 13:42:34 -0800 Subject: [PATCH 130/331] Remove default constructor --- .../twitter/hive/util/TTransportPool.java | 18 ------------------ 1 file changed, 18 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java index 6042a484bfe5..0e34f6756f1c 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java @@ -14,7 +14,6 @@ package com.facebook.presto.twitter.hive.util; import com.google.common.net.HostAndPort; -import io.airlift.units.Duration; import org.apache.commons.pool2.ObjectPool; import org.apache.commons.pool2.PooledObjectFactory; import org.apache.commons.pool2.impl.GenericObjectPool; @@ -23,29 +22,12 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.TimeUnit; public class TTransportPool { - private static final int MIN_IDLE = 0; - private static final int MAX_TOTAL = 128; - private static final int NUM_TESTS = 3; - private static final Duration IDLE_TIMEOUT = new Duration(300, TimeUnit.SECONDS); - private static final Duration EVICTION_INTERVAL = new Duration(10, TimeUnit.SECONDS); private final ConcurrentMap> pools = new ConcurrentHashMap(); private GenericObjectPoolConfig poolConfig; - public TTransportPool() - { - poolConfig = new GenericObjectPoolConfig(); - poolConfig.setMaxIdle(MAX_TOTAL); - poolConfig.setMinIdle(MIN_IDLE); - poolConfig.setMaxTotal(MAX_TOTAL); - poolConfig.setMinEvictableIdleTimeMillis(IDLE_TIMEOUT.toMillis()); - poolConfig.setTimeBetweenEvictionRunsMillis(EVICTION_INTERVAL.toMillis()); - poolConfig.setNumTestsPerEvictionRun(NUM_TESTS); - } - public TTransportPool(GenericObjectPoolConfig poolConfig) { this.poolConfig = poolConfig; From e2bc69238e5fda34e713051d41f076918bcf6fba Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 15 Dec 2016 11:38:16 -0800 Subject: [PATCH 131/331] Apply 100 ruler --- .../PooledHiveMetastoreClientFactory.java | 21 ++++++++++++++----- .../hive/util/PooledTTransportFactory.java | 10 ++++++--- 2 files changed, 23 insertions(+), 8 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java index e411ffa879fe..483a736759b0 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java @@ -38,7 +38,8 @@ public class PooledHiveMetastoreClientFactory private final HiveMetastoreAuthentication metastoreAuthentication; private final TTransportPool transportPool; - public PooledHiveMetastoreClientFactory(@Nullable HostAndPort socksProxy, Duration timeout, HiveMetastoreAuthentication metastoreAuthentication, + public PooledHiveMetastoreClientFactory(@Nullable HostAndPort socksProxy, Duration timeout, + HiveMetastoreAuthentication metastoreAuthentication, int maxTransport, long idleTimeout, long transportEvictInterval, int evictNumTests) { this.socksProxy = socksProxy; @@ -54,10 +55,17 @@ public PooledHiveMetastoreClientFactory(@Nullable HostAndPort socksProxy, Durati } @Inject - public PooledHiveMetastoreClientFactory(HiveClientConfig config, ZookeeperServersetMetastoreConfig zkConfig, HiveMetastoreAuthentication metastoreAuthentication) + public PooledHiveMetastoreClientFactory(HiveClientConfig config, + ZookeeperServersetMetastoreConfig zkConfig, + HiveMetastoreAuthentication metastoreAuthentication) { - this(config.getMetastoreSocksProxy(), config.getMetastoreTimeout(), metastoreAuthentication, - zkConfig.getMaxTransport(), zkConfig.getTransportIdleTimeout(), zkConfig.getTransportEvictInterval(), zkConfig.getTransportEvictNumTests()); + this(config.getMetastoreSocksProxy(), + config.getMetastoreTimeout(), + metastoreAuthentication, + zkConfig.getMaxTransport(), + zkConfig.getTransportIdleTimeout(), + zkConfig.getTransportEvictInterval(), + zkConfig.getTransportEvictNumTests()); } public HiveMetastoreClient create(String host, int port) @@ -66,7 +74,10 @@ public HiveMetastoreClient create(String host, int port) try { TTransport transport = transportPool.borrowObject(host, port); if (transport == null) { - transport = transportPool.borrowObject(host, port, new PooledTTransportFactory(transportPool, host, port, socksProxy, timeoutMillis, metastoreAuthentication)); + transport = transportPool.borrowObject(host, port, + new PooledTTransportFactory(transportPool, + host, port, socksProxy, + timeoutMillis, metastoreAuthentication)); } return new ThriftHiveMetastoreClient(transport); } diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java index b7896754351a..02ac8955c2cd 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java @@ -43,7 +43,9 @@ public class PooledTTransportFactory private final int timeoutMillis; private final HiveMetastoreAuthentication metastoreAuthentication; - public PooledTTransportFactory(TTransportPool pool, String host, int port, @Nullable HostAndPort socksProxy, int timeoutMillis, HiveMetastoreAuthentication metastoreAuthentication) + public PooledTTransportFactory(TTransportPool pool, String host, int port, + @Nullable HostAndPort socksProxy, int timeoutMillis, + HiveMetastoreAuthentication metastoreAuthentication) { this.pool = requireNonNull(pool, "pool is null"); this.host = requireNonNull(host, "host is null"); @@ -68,7 +70,8 @@ public TTransport create() transport = new TSocket(host, port, timeoutMillis); } else { - SocketAddress address = InetSocketAddress.createUnresolved(socksProxy.getHostText(), socksProxy.getPort()); + SocketAddress address = InetSocketAddress.createUnresolved(socksProxy.getHostText(), + socksProxy.getPort()); Socket socket = new Socket(new Proxy(Proxy.Type.SOCKS, address)); try { socket.connect(InetSocketAddress.createUnresolved(host, port), timeoutMillis); @@ -92,7 +95,8 @@ public TTransport create() authenticatedTransport.open(); } - return new PooledTTransport(authenticatedTransport, pool, HostAndPort.fromParts(host, port).toString()); + return new PooledTTransport(authenticatedTransport, pool, + HostAndPort.fromParts(host, port).toString()); } @Override From ae66897529a2765ff154c32d98bcb323108d01b5 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 16 Dec 2016 13:06:31 -0800 Subject: [PATCH 132/331] Update release version to 0.157-tw-0.29 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 46 files changed, 48 insertions(+), 48 deletions(-) diff --git a/pom.xml b/pom.xml index 6458b946f779..4544307a43fa 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.157-tw-0.28 + 0.157-tw-0.29 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index c626c3ad70b2..25b2dfa6873b 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index a500d24dd11f..9464d398187b 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 8761d3638bc6..351ea0688410 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 73936549aa05..d13a25aad0bd 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 4607b9c36eb8..45160e8845c6 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index d7b20168f9d9..4ca2d25d5552 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157-tw-0.28 + 0.157-tw-0.29 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 8ccb6e9fab7a..4660227dace9 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index e5c8a761868a..92241b60b780 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index f49684f0083f..70ce2e31fb2c 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index e2e5a57835bf..784141533e83 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 4f53264a9894..96d2efbf15f1 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 0cffa89f962b..33c7c59d1464 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 533ffeda6853..9d91549fea3f 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 2af0998f777e..2daad579d330 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 03c62da1a4db..d911b3d72990 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 525324657f8d..17f770566608 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 7dfdfd71281c..c0fe52ce0229 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 7eef88e3c083..fb87edef3156 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 059491bbb6f6..d1cb20840cec 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 68f97938c645..8514c25365b5 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index e1d4ffd5735a..8781a46e9ce7 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 9c387bfc64c5..c0fbc160d1c2 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 93da37fd6fc6..97ddd14fac53 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 008c82fdbe20..367ab2fc95d8 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 7087a7cdab75..ff06402a5013 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 874270eb3e39..8519ca166be6 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 40c93e82bd6e..a552f39fabbe 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index a73eefd181a2..578271f897bc 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index e3e52ccc9c94..b54d6eb8a7b6 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 7d6c08ae0f32..646e8ebeb0a9 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 699d996b9cf9..6095c86862f3 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157-tw-0.28 + 0.157-tw-0.29 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index fc953510fc6e..5bd2ec84bd73 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index b550cf7cfe49..62aba5012cb2 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index caa9be15c786..aa4683471ce7 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 33eabb69a5fa..cb0adca83ada 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 0dec7412e1e5..4f975bd1e7e3 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index dedf4daf4cb8..48c5116da66e 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 51b1da095a93..1b0743a1c432 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 8bb450c60706..59eb0ee34a0b 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 2b9ec4feb53f..cca9f667d352 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 8ef0f28df850..0d5b8b0e0a69 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 0923fa884822..1dd7f0988428 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157-tw-0.28 + 0.157-tw-0.29 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 9482bda29427..38b538532789 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 539a9fc3f03f..c6733e132dbe 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index a38d41663d1f..b06141de197f 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.157-tw-0.28 + 0.157-tw-0.29 provided From aba8854b43249d63f19057022fd5021f620d2128 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 20 Dec 2016 11:20:15 -0800 Subject: [PATCH 133/331] invoke isReachable to test connection --- .../hive/util/PooledTTransportFactory.java | 33 +++++++++++++++---- 1 file changed, 26 insertions(+), 7 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java index 02ac8955c2cd..68efa43131cb 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java @@ -55,10 +55,23 @@ public PooledTTransportFactory(TTransportPool pool, String host, int port, this.metastoreAuthentication = requireNonNull(metastoreAuthentication, "metastoreAuthentication is null"); } + @Override + public void activateObject(PooledObject pooledObject) + throws Exception + { + pooledObject.getObject().flush(); + } + @Override public boolean validateObject(PooledObject pooledObject) { - return pooledObject.getObject().isOpen(); + try { + return (pooledObject.getObject().isOpen() && + ((PooledTTransport) pooledObject.getObject()).isReachable(timeoutMillis)); + } + catch (Exception e) { + return false; + } } @Override @@ -147,6 +160,12 @@ public TTransport getTTransport() return transport; } + public boolean isReachable(int timeoutMillis) + throws ClassCastException, IOException + { + return ((TSocket) transport).getSocket().getInetAddress().isReachable(timeoutMillis); + } + @Override public void close() { @@ -196,42 +215,42 @@ public void consumeBuffer(int len) @Override public void open() - throws TTransportException + throws TTransportException { transport.open(); } @Override public int readAll(byte[] bytes, int off, int len) - throws TTransportException + throws TTransportException { return transport.readAll(bytes, off, len); } @Override public int read(byte[] bytes, int off, int len) - throws TTransportException + throws TTransportException { return transport.read(bytes, off, len); } @Override public void write(byte[] bytes) - throws TTransportException + throws TTransportException { transport.write(bytes); } @Override public void write(byte[] bytes, int off, int len) - throws TTransportException + throws TTransportException { transport.write(bytes, off, len); } @Override public void flush() - throws TTransportException + throws TTransportException { transport.flush(); } From b1c48542f3340ae32381722a7c0a6b65da7b7c19 Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Wed, 4 Jan 2017 13:27:36 -0800 Subject: [PATCH 134/331] switch to 0.161 tag first --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 46 files changed, 48 insertions(+), 48 deletions(-) diff --git a/pom.xml b/pom.xml index 4544307a43fa..195c7db584fe 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.157-tw-0.29 + 0.161 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 25b2dfa6873b..2544cc5f1525 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 9464d398187b..2282780a9128 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 351ea0688410..fad7d54f0588 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index d13a25aad0bd..fc8e320103e4 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 45160e8845c6..14777b2334bc 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 4ca2d25d5552..e8fc18894388 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157-tw-0.29 + 0.161 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 4660227dace9..5fd7c56c789d 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 92241b60b780..d997e07f4444 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 70ce2e31fb2c..8f38bb6e8cc2 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 784141533e83..82b574340661 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 96d2efbf15f1..04e58e8133d5 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 33c7c59d1464..793ae8f7cab2 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 9d91549fea3f..710a09b4c3f1 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 2daad579d330..001eb372d4a1 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index d911b3d72990..f0fc34b1e084 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 17f770566608..2244deeae111 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index c0fe52ce0229..ffbda20228fa 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index fb87edef3156..f52e6f7418d1 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index d1cb20840cec..e3ff898e6354 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 8514c25365b5..d3208026e086 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 8781a46e9ce7..69f7007f1706 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index c0fbc160d1c2..7cf3ed1802fa 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 97ddd14fac53..4953177ca686 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 367ab2fc95d8..7defae7dcd90 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index ff06402a5013..cd7f0a2f6558 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 8519ca166be6..fedfd06bb31e 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index a552f39fabbe..eed944e1ea55 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 578271f897bc..870d38cc3bc6 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index b54d6eb8a7b6..e4c62d424c2e 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 646e8ebeb0a9..26bc7d494653 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 6095c86862f3..3578bdedfb46 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157-tw-0.29 + 0.161 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 5bd2ec84bd73..556815dba842 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 62aba5012cb2..3bc78b50778c 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index aa4683471ce7..95de42244f7b 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index cb0adca83ada..45b8de7318c9 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 4f975bd1e7e3..8b2922cbfa0c 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 48c5116da66e..8e1642fff6bd 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 1b0743a1c432..c502c46cb324 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 59eb0ee34a0b..92fe25ab6793 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index cca9f667d352..0f1a27d58293 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 0d5b8b0e0a69..34263e6d85b3 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 1dd7f0988428..c9c564b034ed 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157-tw-0.29 + 0.161 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 38b538532789..685fc6942f52 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index c6733e132dbe..307c17576842 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index b06141de197f..c7bbeb7d0d39 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.157-tw-0.29 + 0.161 provided From dcb3a3948bd96d84a0244d1a7208db78c6b5ffdb Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Wed, 4 Jan 2017 13:31:03 -0800 Subject: [PATCH 135/331] revert back to old twitter tag --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 46 files changed, 48 insertions(+), 48 deletions(-) diff --git a/pom.xml b/pom.xml index 09f571764e7d..1a5bb015822c 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.161 + 0.157-tw-0.28 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 2544cc5f1525..c626c3ad70b2 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 2282780a9128..a500d24dd11f 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index fad7d54f0588..8761d3638bc6 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index fc8e320103e4..73936549aa05 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 14777b2334bc..4607b9c36eb8 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index e8fc18894388..d7b20168f9d9 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.161 + 0.157-tw-0.28 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 5fd7c56c789d..8ccb6e9fab7a 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index d997e07f4444..e5c8a761868a 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 8f38bb6e8cc2..f49684f0083f 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 82b574340661..e2e5a57835bf 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 04e58e8133d5..4f53264a9894 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index d431d1265028..087f040d63ae 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 710a09b4c3f1..533ffeda6853 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 001eb372d4a1..2af0998f777e 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index f0fc34b1e084..03c62da1a4db 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 2244deeae111..525324657f8d 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index ffbda20228fa..7dfdfd71281c 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index f52e6f7418d1..7eef88e3c083 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index e3ff898e6354..059491bbb6f6 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index d3208026e086..68f97938c645 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 69f7007f1706..e1d4ffd5735a 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 7cf3ed1802fa..9c387bfc64c5 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 4953177ca686..93da37fd6fc6 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 7defae7dcd90..008c82fdbe20 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index cd7f0a2f6558..7087a7cdab75 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index fedfd06bb31e..874270eb3e39 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index eed944e1ea55..40c93e82bd6e 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 870d38cc3bc6..a73eefd181a2 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index e4c62d424c2e..e3e52ccc9c94 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 26bc7d494653..7d6c08ae0f32 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 3ad29be3abec..4cedd7b332d3 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.161 + 0.157-tw-0.28 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 556815dba842..fc953510fc6e 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 3bc78b50778c..b550cf7cfe49 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 95de42244f7b..caa9be15c786 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 45b8de7318c9..33eabb69a5fa 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 8b2922cbfa0c..0dec7412e1e5 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 8e1642fff6bd..dedf4daf4cb8 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index c502c46cb324..51b1da095a93 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 92fe25ab6793..8bb450c60706 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 0f1a27d58293..2b9ec4feb53f 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 34263e6d85b3..8ef0f28df850 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 170e4291d73b..d0ea923390cf 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.161 + 0.157-tw-0.28 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 685fc6942f52..9482bda29427 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 307c17576842..539a9fc3f03f 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index c7bbeb7d0d39..a38d41663d1f 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.161 + 0.157-tw-0.28 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.161 + 0.157-tw-0.28 provided From 0b0da52b96c4fc961b575346f8a4e17c2f86cc59 Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Thu, 5 Jan 2017 09:22:31 -0800 Subject: [PATCH 136/331] correct revert version --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 46 files changed, 48 insertions(+), 48 deletions(-) diff --git a/pom.xml b/pom.xml index 1a5bb015822c..a321d331717c 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.157-tw-0.28 + 0.157-tw-0.29 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index c626c3ad70b2..25b2dfa6873b 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index a500d24dd11f..9464d398187b 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 8761d3638bc6..351ea0688410 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 73936549aa05..d13a25aad0bd 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 4607b9c36eb8..45160e8845c6 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index d7b20168f9d9..4ca2d25d5552 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157-tw-0.28 + 0.157-tw-0.29 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 8ccb6e9fab7a..4660227dace9 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index e5c8a761868a..92241b60b780 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index f49684f0083f..70ce2e31fb2c 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index e2e5a57835bf..784141533e83 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 4f53264a9894..96d2efbf15f1 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 087f040d63ae..39d843f4c136 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 533ffeda6853..9d91549fea3f 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 2af0998f777e..2daad579d330 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 03c62da1a4db..d911b3d72990 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 525324657f8d..17f770566608 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 7dfdfd71281c..c0fe52ce0229 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 7eef88e3c083..fb87edef3156 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 059491bbb6f6..d1cb20840cec 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 68f97938c645..8514c25365b5 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index e1d4ffd5735a..8781a46e9ce7 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 9c387bfc64c5..c0fbc160d1c2 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 93da37fd6fc6..97ddd14fac53 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 008c82fdbe20..367ab2fc95d8 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 7087a7cdab75..ff06402a5013 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 874270eb3e39..8519ca166be6 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 40c93e82bd6e..a552f39fabbe 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index a73eefd181a2..578271f897bc 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index e3e52ccc9c94..b54d6eb8a7b6 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 7d6c08ae0f32..646e8ebeb0a9 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 4cedd7b332d3..0225fb57e04e 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157-tw-0.28 + 0.157-tw-0.29 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index fc953510fc6e..5bd2ec84bd73 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index b550cf7cfe49..62aba5012cb2 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index caa9be15c786..aa4683471ce7 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 33eabb69a5fa..cb0adca83ada 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 0dec7412e1e5..4f975bd1e7e3 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index dedf4daf4cb8..48c5116da66e 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 51b1da095a93..1b0743a1c432 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 8bb450c60706..59eb0ee34a0b 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 2b9ec4feb53f..cca9f667d352 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 8ef0f28df850..0d5b8b0e0a69 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index d0ea923390cf..2d8d6e714669 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157-tw-0.28 + 0.157-tw-0.29 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 9482bda29427..38b538532789 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 539a9fc3f03f..c6733e132dbe 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index a38d41663d1f..b06141de197f 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.28 + 0.157-tw-0.29 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.157-tw-0.28 + 0.157-tw-0.29 provided From 4362f3d576d382f124915541c7439147099a294d Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 11 Jan 2017 17:58:28 -0800 Subject: [PATCH 137/331] upgrade version to 0.161-tw-0.29 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 46 files changed, 48 insertions(+), 48 deletions(-) diff --git a/pom.xml b/pom.xml index a321d331717c..0c635940caac 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.157-tw-0.29 + 0.161-tw-0.29 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 25b2dfa6873b..fb7c156dd9a1 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 9464d398187b..dc2a98955890 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 351ea0688410..82c870a7c0a1 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index d13a25aad0bd..da91f29d186b 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 45160e8845c6..4eea6d2f9715 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 4ca2d25d5552..3903f910d7ad 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157-tw-0.29 + 0.161-tw-0.29 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 4660227dace9..7c8199f8f55f 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 92241b60b780..274206c530f8 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 70ce2e31fb2c..5489ef0bbcb3 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 784141533e83..2a2bb8a107d9 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 96d2efbf15f1..d0449e038d57 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 39d843f4c136..1ce39ab1de65 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 9d91549fea3f..c7e73e600add 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 2daad579d330..70cbc172fcf3 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index d911b3d72990..a7f7bf5e3b09 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 17f770566608..32ace12602c5 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index c0fe52ce0229..6a47deee2f8b 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index fb87edef3156..69ccbbe1e0a4 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index d1cb20840cec..2d8285f36a1b 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 8514c25365b5..1994a7a03277 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 8781a46e9ce7..dc963090b629 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index c0fbc160d1c2..016dea4d6570 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 97ddd14fac53..01880b166a2a 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 367ab2fc95d8..d39240e8dfe6 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index ff06402a5013..be884eb7091f 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 8519ca166be6..ef457b874ed9 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index a552f39fabbe..fe301cf1d0f0 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 578271f897bc..6d2293ff28e1 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index b54d6eb8a7b6..662e4d8cc303 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 646e8ebeb0a9..693e05f07c32 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 0225fb57e04e..0fb9fc6dc5cb 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157-tw-0.29 + 0.161-tw-0.29 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 5bd2ec84bd73..d748171603da 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 62aba5012cb2..f7b22d29d943 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index aa4683471ce7..4f4c1f42cd28 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index cb0adca83ada..2993123f9ea7 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 4f975bd1e7e3..f77590c6503d 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 48c5116da66e..6898ef04e08b 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 1b0743a1c432..4730d5453487 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 59eb0ee34a0b..946d3a35ba43 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index cca9f667d352..5553b1d86f81 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 0d5b8b0e0a69..124b35e82b7a 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 2d8d6e714669..fc77811b0752 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157-tw-0.29 + 0.161-tw-0.29 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 38b538532789..2c84563921fa 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index c6733e132dbe..8e640c27df96 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index b06141de197f..953b42916888 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.161-tw-0.29 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.157-tw-0.29 + 0.161-tw-0.29 provided From 575f1806c638f93bc06a6cd3d9ffde58c8db8987 Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Wed, 1 Feb 2017 11:41:18 -0800 Subject: [PATCH 138/331] warn instead of fail when JVM is non-Oracle --- .../com/facebook/presto/server/PrestoSystemRequirements.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-main/src/main/java/com/facebook/presto/server/PrestoSystemRequirements.java b/presto-main/src/main/java/com/facebook/presto/server/PrestoSystemRequirements.java index fdf7bc5f1031..de32c7d23571 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/PrestoSystemRequirements.java +++ b/presto-main/src/main/java/com/facebook/presto/server/PrestoSystemRequirements.java @@ -42,7 +42,7 @@ public static void verifyJvmRequirements() { String vendor = StandardSystemProperty.JAVA_VENDOR.value(); if (!"Oracle Corporation".equals(vendor)) { - failRequirement("Presto requires an Oracle or OpenJDK JVM (found %s)", vendor); + warnRequirement("Presto requires an Oracle or OpenJDK JVM (found %s)", vendor); } //verifyJavaVersion(); From 656f1494597abf438f225dae2cfda7147c550054 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 2 Feb 2017 15:45:23 -0800 Subject: [PATCH 139/331] Revert "Merge pull request #67 from dabaitu/switchJVM" This reverts commit 5dd9eda40897eaab492b0b0118805bf750463030, reversing changes made to 8d4e86ae5915f5a5c825fbebf13a21ee42186048. --- .../com/facebook/presto/server/PrestoSystemRequirements.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-main/src/main/java/com/facebook/presto/server/PrestoSystemRequirements.java b/presto-main/src/main/java/com/facebook/presto/server/PrestoSystemRequirements.java index de32c7d23571..fdf7bc5f1031 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/PrestoSystemRequirements.java +++ b/presto-main/src/main/java/com/facebook/presto/server/PrestoSystemRequirements.java @@ -42,7 +42,7 @@ public static void verifyJvmRequirements() { String vendor = StandardSystemProperty.JAVA_VENDOR.value(); if (!"Oracle Corporation".equals(vendor)) { - warnRequirement("Presto requires an Oracle or OpenJDK JVM (found %s)", vendor); + failRequirement("Presto requires an Oracle or OpenJDK JVM (found %s)", vendor); } //verifyJavaVersion(); From ebf8b53b1d5bc11fcabfa0d42c83262b6e2b5b12 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 2 Feb 2017 15:46:44 -0800 Subject: [PATCH 140/331] Revert "Merge pull request #65 from Yaliang/yaliangw/upgradeVersion" This reverts commit 8d4e86ae5915f5a5c825fbebf13a21ee42186048, reversing changes made to 76f4364057fc201e18478b6ca6c5b08f7811baba. --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 46 files changed, 48 insertions(+), 48 deletions(-) diff --git a/pom.xml b/pom.xml index 0c635940caac..a321d331717c 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.161-tw-0.29 + 0.157-tw-0.29 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index fb7c156dd9a1..25b2dfa6873b 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index dc2a98955890..9464d398187b 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 82c870a7c0a1..351ea0688410 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index da91f29d186b..d13a25aad0bd 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 4eea6d2f9715..45160e8845c6 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 3903f910d7ad..4ca2d25d5552 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.161-tw-0.29 + 0.157-tw-0.29 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 7c8199f8f55f..4660227dace9 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 274206c530f8..92241b60b780 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 5489ef0bbcb3..70ce2e31fb2c 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 2a2bb8a107d9..784141533e83 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index d0449e038d57..96d2efbf15f1 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 1ce39ab1de65..39d843f4c136 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index c7e73e600add..9d91549fea3f 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 70cbc172fcf3..2daad579d330 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index a7f7bf5e3b09..d911b3d72990 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 32ace12602c5..17f770566608 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 6a47deee2f8b..c0fe52ce0229 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 69ccbbe1e0a4..fb87edef3156 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 2d8285f36a1b..d1cb20840cec 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 1994a7a03277..8514c25365b5 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index dc963090b629..8781a46e9ce7 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 016dea4d6570..c0fbc160d1c2 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 01880b166a2a..97ddd14fac53 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index d39240e8dfe6..367ab2fc95d8 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index be884eb7091f..ff06402a5013 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index ef457b874ed9..8519ca166be6 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index fe301cf1d0f0..a552f39fabbe 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 6d2293ff28e1..578271f897bc 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 662e4d8cc303..b54d6eb8a7b6 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 693e05f07c32..646e8ebeb0a9 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 0fb9fc6dc5cb..0225fb57e04e 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.161-tw-0.29 + 0.157-tw-0.29 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index d748171603da..5bd2ec84bd73 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index f7b22d29d943..62aba5012cb2 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 4f4c1f42cd28..aa4683471ce7 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 2993123f9ea7..cb0adca83ada 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index f77590c6503d..4f975bd1e7e3 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 6898ef04e08b..48c5116da66e 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 4730d5453487..1b0743a1c432 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 946d3a35ba43..59eb0ee34a0b 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 5553b1d86f81..cca9f667d352 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 124b35e82b7a..0d5b8b0e0a69 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index fc77811b0752..2d8d6e714669 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.161-tw-0.29 + 0.157-tw-0.29 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 2c84563921fa..38b538532789 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 8e640c27df96..c6733e132dbe 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 953b42916888..b06141de197f 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.161-tw-0.29 + 0.157-tw-0.29 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.161-tw-0.29 + 0.157-tw-0.29 provided From 1bca4873be34375203cd57807cacd5fce7864a98 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 2 Feb 2017 15:49:56 -0800 Subject: [PATCH 141/331] Revert "Merge pull request #62 from Yaliang/yaliangw/connectionValidate" This reverts commit 76f4364057fc201e18478b6ca6c5b08f7811baba, reversing changes made to 0cf760de30943d72f14b467b099a55cb3cfba14b. --- .../hive/util/PooledTTransportFactory.java | 33 ++++--------------- 1 file changed, 7 insertions(+), 26 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java index 68efa43131cb..02ac8955c2cd 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java @@ -55,23 +55,10 @@ public PooledTTransportFactory(TTransportPool pool, String host, int port, this.metastoreAuthentication = requireNonNull(metastoreAuthentication, "metastoreAuthentication is null"); } - @Override - public void activateObject(PooledObject pooledObject) - throws Exception - { - pooledObject.getObject().flush(); - } - @Override public boolean validateObject(PooledObject pooledObject) { - try { - return (pooledObject.getObject().isOpen() && - ((PooledTTransport) pooledObject.getObject()).isReachable(timeoutMillis)); - } - catch (Exception e) { - return false; - } + return pooledObject.getObject().isOpen(); } @Override @@ -160,12 +147,6 @@ public TTransport getTTransport() return transport; } - public boolean isReachable(int timeoutMillis) - throws ClassCastException, IOException - { - return ((TSocket) transport).getSocket().getInetAddress().isReachable(timeoutMillis); - } - @Override public void close() { @@ -215,42 +196,42 @@ public void consumeBuffer(int len) @Override public void open() - throws TTransportException + throws TTransportException { transport.open(); } @Override public int readAll(byte[] bytes, int off, int len) - throws TTransportException + throws TTransportException { return transport.readAll(bytes, off, len); } @Override public int read(byte[] bytes, int off, int len) - throws TTransportException + throws TTransportException { return transport.read(bytes, off, len); } @Override public void write(byte[] bytes) - throws TTransportException + throws TTransportException { transport.write(bytes); } @Override public void write(byte[] bytes, int off, int len) - throws TTransportException + throws TTransportException { transport.write(bytes, off, len); } @Override public void flush() - throws TTransportException + throws TTransportException { transport.flush(); } From 79382a2894f5f4514c7eda60fbfcaa1f3e8b69c6 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 2 Feb 2017 15:51:40 -0800 Subject: [PATCH 142/331] Revert "Merge pull request #63 from dabaitu/twitter-master" This reverts commit 0cf760de30943d72f14b467b099a55cb3cfba14b, reversing changes made to c1c4575c0eefe6845876910954a2db62309c6010. --- .travis.yml | 33 +- README.md | 2 +- pom.xml | 12 +- .../presto/plugin/jdbc/MetadataUtil.java | 5 +- .../jdbc/TestJdbcOutputTableHandle.java | 3 +- .../presto/plugin/jdbc/TestJdbcRecordSet.java | 2 +- .../jdbc/TestJdbcRecordSetProvider.java | 24 +- .../presto/plugin/jdbc/TestJdbcSplit.java | 5 +- .../presto/plugin/jdbc/TestingDatabase.java | 3 +- .../benchmark/driver/BenchmarkSchema.java | 2 +- .../presto/benchmark/driver/Suite.java | 2 +- .../benchmark/AbstractOperatorBenchmark.java | 2 +- .../benchmark/BenchmarkQueryRunner.java | 2 +- .../presto/benchmark/BenchmarkSuite.java | 2 +- .../presto/benchmark/HandTpchQuery1.java | 2 +- .../presto/benchmark/HandTpchQuery6.java | 2 +- .../benchmark/HashBuildAndJoinBenchmark.java | 5 +- .../benchmark/PredicateFilterBenchmark.java | 3 +- .../blackhole/BlackHoleConnectorFactory.java | 2 +- .../plugin/blackhole/BlackHolePageSink.java | 16 +- .../plugin/blackhole/BlackHolePageSource.java | 125 -- .../BlackHolePageSourceProvider.java | 16 +- .../blackhole/BlackHoleSplitManager.java | 2 +- .../plugin/blackhole/DelayPageSource.java | 55 + .../facebook/presto/bytecode/ArrayOpCode.java | 2 +- .../presto/bytecode/ClassDefinition.java | 2 +- .../presto/bytecode/ClassInfoLoader.java | 2 +- .../facebook/presto/bytecode/Variable.java | 2 +- .../presto/bytecode/control/DoWhileLoop.java | 2 +- .../expression/AndBytecodeExpression.java | 2 +- .../ArithmeticBytecodeExpression.java | 2 +- .../ArrayLengthBytecodeExpression.java | 2 +- .../expression/CastBytecodeExpression.java | 2 +- .../ComparisonBytecodeExpression.java | 2 +- .../GetElementBytecodeExpression.java | 2 +- .../GetFieldBytecodeExpression.java | 2 +- .../InlineIfBytecodeExpression.java | 2 +- .../expression/NegateBytecodeExpression.java | 2 +- .../NewInstanceBytecodeExpression.java | 2 +- .../expression/NotBytecodeExpression.java | 2 +- .../expression/OrBytecodeExpression.java | 2 +- .../expression/PopBytecodeExpression.java | 2 +- .../expression/ReturnBytecodeExpression.java | 2 +- .../SetArrayElementBytecodeExpression.java | 2 +- .../cassandra/CassandraClientModule.java | 2 +- .../presto/cassandra/CassandraSession.java | 2 - .../presto/cassandra/RetryDriver.java | 2 +- .../cassandra/MockCassandraSession.java | 1 - .../cassandra/TestCassandraDistributed.java | 20 +- .../cassandra/TestJsonCassandraHandles.java | 2 +- .../util/TestHostAddressFactory.java | 2 +- .../presto/cli/AlignedTablePrinter.java | 2 +- .../facebook/presto/cli/ClientOptions.java | 4 +- .../com/facebook/presto/cli/CsvPrinter.java | 2 +- .../com/facebook/presto/cli/PerfTest.java | 2 +- .../com/facebook/presto/cli/QueryRunner.java | 6 +- .../com/facebook/presto/cli/TsvPrinter.java | 4 +- .../presto/cli/TestTableNameCompleter.java | 15 +- presto-docs/Makefile | 2 +- presto-docs/pom.xml | 2 +- .../src/main/sphinx/connector/accumulo.rst | 34 +- .../src/main/sphinx/connector/cassandra.rst | 10 +- .../main/sphinx/connector/hive-security.rst | 4 +- .../src/main/sphinx/connector/hive.rst | 2 +- presto-docs/src/main/sphinx/connector/jmx.rst | 12 +- .../main/sphinx/connector/kafka-tutorial.rst | 100 +- presto-docs/src/main/sphinx/functions.rst | 1 - .../src/main/sphinx/functions/aggregate.rst | 8 +- .../src/main/sphinx/functions/array.rst | 15 - .../src/main/sphinx/functions/lambda.rst | 74 -- presto-docs/src/main/sphinx/functions/map.rst | 12 +- .../src/main/sphinx/language/types.rst | 6 +- presto-docs/src/main/sphinx/release.rst | 5 - .../main/sphinx/release/release-0.157.1.rst | 9 - .../src/main/sphinx/release/release-0.158.rst | 36 - .../src/main/sphinx/release/release-0.159.rst | 15 - .../src/main/sphinx/release/release-0.160.rst | 22 - .../src/main/sphinx/release/release-0.161.rst | 37 - .../presto/example/ExampleHttpServer.java | 2 +- .../facebook/presto/example/MetadataUtil.java | 5 +- .../presto/example/TestExampleRecordSet.java | 2 +- .../hive/BackgroundHiveSplitLoader.java | 47 +- .../hive/ColumnarBinaryHiveRecordCursor.java | 3 +- .../presto/hive/HdfsConfigurationUpdater.java | 4 - .../facebook/presto/hive/HiveBucketing.java | 44 +- .../presto/hive/HiveClientConfig.java | 60 +- .../presto/hive/HiveClientModule.java | 2 - .../presto/hive/HiveColumnHandle.java | 16 - .../facebook/presto/hive/HiveErrorCode.java | 4 +- .../facebook/presto/hive/HiveMetadata.java | 89 +- .../presto/hive/HiveMetadataFactory.java | 11 +- .../presto/hive/HivePageSourceProvider.java | 10 +- .../facebook/presto/hive/HivePartition.java | 30 +- .../presto/hive/HivePartitionManager.java | 60 +- .../presto/hive/HiveSessionProperties.java | 11 - .../presto/hive/HiveSplitManager.java | 8 +- .../com/facebook/presto/hive/HiveType.java | 23 +- .../com/facebook/presto/hive/HiveUtil.java | 26 +- .../facebook/presto/hive/HiveWriteUtils.java | 15 +- .../hive/PartitionOfflineException.java | 32 +- .../presto/hive/TableOfflineException.java | 24 +- .../hive/metastore/BridgingHiveMetastore.java | 5 + .../hive/metastore/CachingHiveMetastore.java | 39 +- .../hive/metastore/ExtendedHiveMetastore.java | 2 + .../presto/hive/metastore/HiveMetastore.java | 4 + .../hive/metastore/ThriftHiveMetastore.java | 28 +- .../hive/parquet/ParquetPageSource.java | 52 +- .../parquet/ParquetPageSourceFactory.java | 28 +- .../presto/hive/parquet/ParquetTypeUtils.java | 57 - .../parquet/reader/ParquetColumnReader.java | 16 +- .../hive/parquet/reader/ParquetReader.java | 86 +- .../security/UserGroupInformationShim.java | 52 + .../presto/hive/AbstractTestHiveClient.java | 50 +- .../hive/AbstractTestHiveFileFormats.java | 6 +- .../presto/hive/HiveBenchmarkQueryRunner.java | 2 +- .../presto/hive/TestHiveClientConfig.java | 16 +- .../hive/TestHiveDistributedQueries.java | 29 - .../presto/hive/TestHiveFileFormats.java | 6 +- .../hive/TestHiveIntegrationSmokeTest.java | 125 +- .../facebook/presto/hive/TestHiveSplit.java | 2 +- .../presto/hive/TestJsonHiveHandles.java | 2 +- .../hive/TestOrcPageSourceMemoryTracking.java | 2 - .../hive/TestPartitionOfflineException.java | 41 - .../hive/TestTableOfflineException.java | 41 - .../hive/metastore/InMemoryHiveMetastore.java | 5 + .../metastore/TestCachingHiveMetastore.java | 5 +- .../presto/hive/parquet/ParquetTester.java | 9 +- .../connector/jmx/JmxConnectorConfig.java | 17 +- .../connector/jmx/JmxRecordSetProvider.java | 4 +- .../connector/jmx/TestJmxConnectorConfig.java | 57 - .../connector/jmx/TestJmxHistoricalData.java | 12 +- .../kafka/KafkaTableDescriptionSupplier.java | 4 +- .../presto/kafka/util/CodecSupplier.java | 3 +- .../presto/localfile/DataLocation.java | 2 +- .../presto/localfile/MetadataUtil.java | 3 +- .../java/com/facebook/presto/Session.java | 2 +- .../presto/SystemSessionProperties.java | 11 - .../InformationSchemaMetadata.java | 1 - .../InformationSchemaPageSourceProvider.java | 5 +- .../presto/event/query/QueryMonitor.java | 2 +- .../presto/execution/CreateViewTask.java | 6 +- .../presto/execution/FutureStateChange.java | 76 -- .../presto/execution/QueryStateMachine.java | 11 +- .../presto/execution/SqlQueryExecution.java | 2 +- .../execution/SqlQueryQueueManager.java | 2 +- .../execution/StartTransactionTask.java | 4 +- .../presto/execution/StateMachine.java | 36 + .../facebook/presto/execution/TaskInfo.java | 2 +- .../presto/execution/buffer/BufferResult.java | 2 +- .../execution/buffer/OutputBufferInfo.java | 5 - .../resourceGroups/InternalResourceGroup.java | 2 +- .../execution/scheduler/NodeScheduler.java | 5 - .../facebook/presto/memory/QueryContext.java | 8 - .../presto/metadata/FunctionRegistry.java | 132 +- .../presto/metadata/InMemoryNodeManager.java | 2 +- .../metadata/PolymorphicScalarFunction.java | 6 +- .../facebook/presto/metadata/Signature.java | 4 +- .../presto/metadata/SignatureBinder.java | 187 +-- .../presto/operator/DeleteOperator.java | 2 +- .../operator/HashAggregationOperator.java | 176 +-- .../presto/operator/HttpPageBufferClient.java | 2 +- .../presto/operator/MergeHashSort.java | 6 +- .../operator/MetadataDeleteOperator.java | 2 +- .../operator/NestedLoopJoinOperator.java | 2 - .../presto/operator/PageSourceOperator.java | 10 - .../facebook/presto/operator/PagesIndex.java | 2 +- .../presto/operator/PagesIndexOrdering.java | 2 +- .../presto/operator/PipelineStats.java | 2 +- .../ScanFilterAndProjectOperator.java | 11 +- .../presto/operator/TableFinishOperator.java | 2 +- .../presto/operator/TableScanOperator.java | 11 +- .../presto/operator/TableWriterOperator.java | 2 +- .../facebook/presto/operator/TaskStats.java | 4 +- .../presto/operator/WindowOperator.java | 4 +- .../AbstractMinMaxAggregationFunction.java | 14 +- .../aggregation/AbstractMinMaxBy.java | 2 +- .../AbstractMinMaxByNAggregationFunction.java | 2 +- .../AbstractMinMaxNAggregationFunction.java | 3 +- .../aggregation/AccumulatorCompiler.java | 6 +- .../aggregation/AggregationCompiler.java | 2 +- .../ApproximateSetAggregation.java | 2 +- .../ArbitraryAggregationFunction.java | 14 +- .../aggregation/ArrayAggregationFunction.java | 2 +- .../BindableAggregationFunction.java | 4 +- .../ChecksumAggregationFunction.java | 6 +- .../operator/aggregation/CountColumn.java | 6 +- .../DecimalAverageAggregation.java | 2 +- .../aggregation/DecimalSumAggregation.java | 2 +- .../operator/aggregation/Histogram.java | 2 +- .../LazyAccumulatorFactoryBinder.java | 2 +- .../aggregation/MapAggregationFunction.java | 2 +- .../aggregation/MapUnionAggregation.java | 2 +- .../MergeHyperLogLogAggregation.java | 2 +- .../MultimapAggregationFunction.java | 2 +- .../builder/HashAggregationBuilder.java | 5 - .../InMemoryHashAggregationBuilder.java | 209 +--- .../MergingHashAggregationBuilder.java | 136 -- .../SpillableHashAggregationBuilder.java | 273 ---- .../aggregation/state/StateCompiler.java | 16 +- .../presto/operator/scalar/ApplyFunction.java | 97 -- .../scalar/ArrayElementAtFunction.java | 21 +- .../operator/scalar/ArrayFilterFunction.java | 105 -- .../operator/scalar/ArrayReduceFunction.java | 115 -- .../scalar/ArrayTransformFunction.java | 104 -- .../operator/scalar/EmptyMapConstructor.java | 39 - .../operator/scalar/FailureFunction.java | 2 +- .../operator/scalar/MapConcatFunction.java | 7 - .../operator/scalar/MapFilterFunction.java | 109 -- .../presto/server/CoordinatorModule.java | 9 + .../presto/server/ExecuteResource.java | 178 +++ .../facebook/presto/server/ForExecute.java | 22 +- .../facebook/presto/server/PluginManager.java | 8 + .../facebook/presto/server/QueryResource.java | 47 +- .../facebook/presto/server/ResourceUtil.java | 58 +- .../presto/server/ServerMainModule.java | 5 +- .../presto/server/StatementResource.java | 14 +- .../facebook/presto/server/TaskResource.java | 12 +- .../presto/server/ThreadResource.java | 2 +- .../ContinuousTaskStatusFetcher.java | 3 - .../server/remotetask/HttpRemoteTask.java | 63 +- .../presto/server/security/SpnegoFilter.java | 23 +- .../server/testing/TestingPrestoServer.java | 4 +- .../presto/spiller/BinaryFileSpiller.java | 27 +- .../presto/spiller/BinarySpillerFactory.java | 4 +- .../presto/spiller/SpillerFactory.java | 4 - .../facebook/presto/sql/ExpressionUtils.java | 7 - .../sql/analyzer/AggregationAnalyzer.java | 15 +- .../presto/sql/analyzer/Analysis.java | 18 - .../sql/analyzer/ExpressionAnalysis.java | 13 +- .../sql/analyzer/ExpressionAnalyzer.java | 291 ++--- .../presto/sql/analyzer/FeaturesConfig.java | 13 - .../presto/sql/analyzer/QueryExplainer.java | 2 +- .../presto/sql/analyzer/RelationType.java | 5 +- .../facebook/presto/sql/analyzer/Scope.java | 4 +- .../sql/analyzer/SemanticErrorCode.java | 2 - .../sql/analyzer/StatementAnalyzer.java | 315 +---- .../sql/analyzer/TypeSignatureProvider.java | 87 -- .../sql/gen/BytecodeExpressionVisitor.java | 31 +- .../sql/gen/CursorProcessorCompiler.java | 137 +-- .../presto/sql/gen/InCodeGenerator.java | 2 +- .../sql/gen/InputReferenceCompiler.java | 18 +- .../facebook/presto/sql/gen/JoinCompiler.java | 2 +- .../sql/gen/JoinFilterFunctionCompiler.java | 122 +- .../presto/sql/gen/JoinProbeCompiler.java | 16 +- .../gen/LambdaAndTryExpressionExtractor.java | 120 -- .../sql/gen/LambdaBytecodeGenerator.java | 183 --- .../presto/sql/gen/NullIfCodeGenerator.java | 2 +- .../presto/sql/gen/OrderingCompiler.java | 2 +- .../presto/sql/gen/PageProcessorCompiler.java | 118 +- .../presto/sql/gen/ParameterAndType.java | 41 - .../sql/gen/PreGeneratedExpressions.java | 47 - .../presto/sql/gen/TryCodeGenerator.java | 20 +- .../sql/gen/TryExpressionExtractor.java | 70 ++ .../sql/gen/VarArgsToMapAdapterGenerator.java | 93 -- .../presto/sql/planner/DomainTranslator.java | 2 +- .../planner/EffectivePredicateExtractor.java | 2 +- .../sql/planner/ExpressionInterpreter.java | 75 +- .../sql/planner/ExpressionSymbolInliner.java | 8 - .../InterpretedInternalFilterFunction.java | 3 +- .../InterpretedProjectionFunction.java | 3 +- .../sql/planner/LiteralInterpreter.java | 10 +- .../sql/planner/LocalExecutionPlanner.java | 29 +- .../presto/sql/planner/LogicalPlanner.java | 55 +- .../presto/sql/planner/PlanBuilder.java | 2 +- .../presto/sql/planner/PlanFragmenter.java | 7 +- .../presto/sql/planner/PlanOptimizers.java | 12 +- .../presto/sql/planner/PlanPrinter.java | 53 +- .../presto/sql/planner/QueryPlanner.java | 43 +- .../presto/sql/planner/RelationPlanner.java | 55 +- .../sql/planner/StageExecutionPlan.java | 2 +- .../presto/sql/planner/SubqueryPlanner.java | 259 ++-- .../sql/planner/SymbolToInputRewriter.java | 40 +- .../presto/sql/planner/TranslationMap.java | 37 +- .../planner/optimizations/AddExchanges.java | 1 - .../CanonicalizeExpressions.java | 10 +- .../optimizations/EvaluateConstantApply.java | 84 ++ .../optimizations/ExpressionEquivalence.java | 17 +- .../HashGenerationOptimizer.java | 2 +- .../ImplementSampleAsFilter.java | 3 +- .../planner/optimizations/MergeWindows.java | 1 - .../MetadataDeleteOptimizer.java | 2 +- .../optimizations/MetadataQueryOptimizer.java | 4 +- .../OptimizeMixedDistinctAggregations.java | 11 +- .../optimizations/PredicatePushDown.java | 40 +- .../optimizations/PreferredProperties.java | 2 +- .../optimizations/PropertyDerivations.java | 2 +- .../PruneUnreferencedOutputs.java | 200 ++- ...lyNodes.java => RemoveRedundantApply.java} | 39 +- .../optimizations/ScalarQueryUtil.java | 14 - .../optimizations/SetFlatteningOptimizer.java | 5 +- .../optimizations/SimplifyExpressions.java | 6 +- ...formCorrelatedScalarAggregationToJoin.java | 13 +- .../TransformExistsApplyToScalarApply.java | 135 -- ...uantifiedComparisonApplyToScalarApply.java | 243 ---- ...rrelatedInPredicateSubqueryToSemiJoin.java | 210 +++- .../TransformUncorrelatedScalarToJoin.java | 3 +- .../UnaliasSymbolReferences.java | 170 +-- .../sql/planner/plan/AggregationNode.java | 20 - .../presto/sql/planner/plan/ApplyNode.java | 48 +- .../sql/planner/plan/ChildReplacer.java | 2 +- .../presto/sql/planner/plan/ExchangeNode.java | 1 - .../presto/sql/planner/plan/JoinNode.java | 24 - .../sql/planner/sanity/PlanSanityChecker.java | 3 +- .../sanity/ValidateDependenciesChecker.java | 12 +- .../sanity/VerifyOnlyOneOutputNode.java | 48 - .../sql/relational/DeterminismEvaluator.java | 12 - .../presto/sql/relational/Expressions.java | 15 - .../LambdaDefinitionExpression.java | 95 -- .../sql/relational/RowExpressionVisitor.java | 2 - .../SqlToRowExpressionTranslator.java | 38 +- .../VariableReferenceExpression.java | 75 -- .../optimizer/ExpressionOptimizer.java | 15 - .../sql/rewrite/DescribeInputRewrite.java | 2 +- .../sql/rewrite/DescribeOutputRewrite.java | 2 +- .../sql/rewrite/ShowQueriesRewrite.java | 1 - .../presto/testing/LocalQueryRunner.java | 16 +- .../testing/TestingEventListenerManager.java | 3 - .../facebook/presto/testing/TestingSplit.java | 4 +- .../presto/transaction/InternalConnector.java | 1 - .../facebook/presto/type/BigintOperators.java | 14 - .../facebook/presto/type/CharOperators.java | 11 +- .../facebook/presto/type/DecimalCasts.java | 105 +- .../type/DecimalSaturatedFloorCasts.java | 162 +-- .../facebook/presto/type/DoubleOperators.java | 70 +- .../presto/type/IntegerOperators.java | 15 - .../com/facebook/presto/type/Re2JRegexp.java | 2 +- .../facebook/presto/type/RealOperators.java | 52 - .../presto/type/SmallintOperators.java | 8 - .../facebook/presto/type/TypeRegistry.java | 7 +- .../presto/type/VarcharOperators.java | 28 +- .../presto/type/VarcharParametricType.java | 4 +- .../facebook/presto/TestHiddenColumns.java | 10 +- .../presto/block/AbstractTestBlock.java | 3 - .../facebook/presto/block/TestArrayBlock.java | 6 +- .../facebook/presto/block/TestPagesSerde.java | 2 +- .../execution/MockRemoteTaskFactory.java | 5 +- .../presto/execution/TaskTestUtils.java | 3 +- .../presto/execution/TestSqlTask.java | 11 +- .../presto/execution/TestSqlTaskManager.java | 11 +- .../execution/TestStageStateMachine.java | 3 +- .../resourceGroups/TestResourceGroups.java | 2 +- .../TestPhasedExecutionSchedule.java | 11 +- .../TestSourcePartitionedScheduler.java | 7 +- .../TestHeartbeatFailureDetector.java | 2 +- .../presto/memory/TestMemoryPools.java | 2 +- .../presto/metadata/TestFunctionRegistry.java | 11 +- .../TestInformationSchemaTableHandle.java | 2 +- .../presto/metadata/TestSignature.java | 3 +- .../presto/metadata/TestSignatureBinder.java | 125 +- .../presto/operator/BenchmarkGroupByHash.java | 6 +- .../BenchmarkHashBuildAndJoinOperators.java | 1 - .../MockExchangeRequestProcessor.java | 2 +- .../facebook/presto/operator/TestDriver.java | 12 +- .../presto/operator/TestExchangeOperator.java | 8 +- .../TestFilterAndProjectOperator.java | 2 +- .../operator/TestHashAggregationOperator.java | 211 +--- .../presto/operator/TestHashJoinOperator.java | 42 +- .../operator/TestHashSemiJoinOperator.java | 16 +- .../operator/TestHttpPageBufferClient.java | 4 +- .../presto/operator/TestMergeHashSort.java | 2 +- .../operator/TestNestedLoopJoinOperator.java | 56 +- .../operator/TestRowNumberOperator.java | 5 +- .../TestScanFilterAndProjectOperator.java | 19 +- .../operator/TestTopNRowNumberOperator.java | 3 +- .../presto/operator/TestUnnestOperator.java | 8 +- .../presto/operator/TestWindowOperator.java | 2 +- .../AbstractTestAggregationFunction.java | 5 +- ...AbstractTestDecimalAverageAggregation.java | 2 +- .../AbstractTestDecimalSumAggregation.java | 2 +- .../aggregation/TestArrayMaxNAggregation.java | 4 +- .../TestDoubleHistogramAggregation.java | 2 +- .../aggregation/TestMinMaxByAggregation.java | 2 +- .../TestRealHistogramAggregation.java | 2 +- .../aggregation/TestStateCompiler.java | 39 +- .../operator/aggregation/TestTypedHeap.java | 2 +- .../aggregation/TestTypedKeyValueHeap.java | 2 +- .../TestBigIntegerAndLongStateSerializer.java | 2 +- .../state/TestBigIntegerStateSerializer.java | 2 +- .../index/TestTupleFilterProcessor.java | 4 +- .../operator/scalar/BenchmarkArrayFilter.java | 268 ---- .../scalar/BenchmarkArraySubscript.java | 8 +- .../operator/scalar/BenchmarkMapConcat.java | 236 ---- .../scalar/BenchmarkMapSubscript.java | 6 +- .../operator/scalar/FunctionAssertions.java | 16 +- .../operator/scalar/TestApplyFunction.java | 155 --- .../scalar/TestArrayFilterFunction.java | 76 -- .../scalar/TestArrayReduceFunction.java | 98 -- .../scalar/TestArrayTransformFunction.java | 135 -- .../scalar/TestMapFilterFunction.java | 197 --- .../operator/scalar/TestStringFunctions.java | 2 +- .../security/TestAccessControlManager.java | 2 +- .../presto/server/TestExecuteResource.java | 90 ++ .../server/remotetask/TestHttpRemoteTask.java | 232 ---- .../presto/spiller/TestBinaryFileSpiller.java | 6 +- .../presto/sql/TestExpressionInterpreter.java | 2 +- .../presto/sql/analyzer/TestAnalyzer.java | 14 +- .../sql/analyzer/TestFeaturesConfig.java | 8 +- .../presto/sql/analyzer/TestScope.java | 2 +- .../sql/gen/PageProcessorBenchmark.java | 5 +- .../sql/gen/TestExpressionCompiler.java | 20 +- .../presto/sql/gen/TestJoinCompiler.java | 8 +- .../presto/sql/gen/TestJoinProbeCompiler.java | 4 +- .../sql/planner/TestDomainTranslator.java | 1096 ++++++++--------- .../TestEffectivePredicateExtractor.java | 14 +- .../sql/planner/TestEqualityInference.java | 2 +- .../TestInterpretedFilterFunction.java | 7 +- .../TestInterpretedProjectionFunction.java | 20 +- .../sql/planner/TestLogicalPlanner.java | 290 ++++- .../planner/TestPlanMatchingFramework.java | 200 --- .../sql/planner/TestPredicatePushdown.java | 46 - .../sql/planner/TestQuantifiedComparison.java | 131 -- .../sql/planner/TestingTableHandle.java | 8 - .../AggregationFunctionMatcher.java | 65 - .../assertions/AggregationMatcher.java | 36 +- .../presto/sql/planner/assertions/Alias.java | 68 - .../sql/planner/assertions/AnySymbol.java | 80 -- .../assertions/AnySymbolReference.java | 50 - .../sql/planner/assertions/BasePlanTest.java | 114 -- .../planner/assertions/ColumnReference.java | 93 -- .../assertions/CorrelationMatcher.java | 34 +- .../assertions/EquiJoinClauseProvider.java | 36 - .../assertions/ExpectedValueProvider.java | 33 - .../planner/assertions/ExpressionAliases.java | 71 ++ .../planner/assertions/ExpressionMatcher.java | 99 -- .../assertions/ExpressionVerifier.java | 106 +- .../sql/planner/assertions/FilterMatcher.java | 21 +- .../assertions/FunctionCallProvider.java | 122 -- .../planner/assertions/GroupIdMatcher.java | 23 +- .../sql/planner/assertions/JoinMatcher.java | 70 +- .../sql/planner/assertions/MatchResult.java | 65 - .../sql/planner/assertions/Matcher.java | 50 +- .../assertions/NotPlanNodeMatcher.java | 11 +- .../sql/planner/assertions/OutputMatcher.java | 74 -- .../sql/planner/assertions/PlanAssert.java | 9 +- .../planner/assertions/PlanMatchPattern.java | 331 ++--- ...bolAlias.java => PlanMatchingContext.java} | 29 +- .../planner/assertions/PlanMatchingState.java | 23 +- .../assertions/PlanMatchingVisitor.java | 142 +-- .../planner/assertions/PlanNodeMatcher.java | 11 +- .../planner/assertions/PlanTestSymbol.java | 21 - .../sql/planner/assertions/RvalueMatcher.java | 36 - .../planner/assertions/SemiJoinMatcher.java | 26 +- .../assertions/SpecificationProvider.java | 73 -- .../sql/planner/assertions/SymbolAliases.java | 232 ---- .../sql/planner/assertions/SymbolMatcher.java | 63 + .../planner/assertions/TableScanMatcher.java | 24 +- .../assertions/TestExpressionVerifier.java | 27 +- .../sql/planner/assertions/ValuesMatcher.java | 45 - .../assertions/WindowFunctionMatcher.java | 65 - .../sql/planner/assertions/WindowMatcher.java | 51 +- .../optimizations/TestMergeWindows.java | 398 +++--- ...TestMixedDistinctAggregationOptimizer.java | 22 +- .../TestSetFlatteningOptimizer.java | 132 -- .../sanity/TestVerifyOnlyOneOutputNode.java | 69 -- .../presto/type/TestArrayOperators.java | 14 +- .../presto/type/TestCharOperators.java | 40 - .../presto/type/TestDecimalCasts.java | 14 - .../presto/type/TestMapOperators.java | 8 +- .../presto/type/TestRowOperators.java | 2 +- .../presto/type/TestTypeRegistry.java | 2 +- .../presto/util/TestPowerOfTwoValidator.java | 2 +- .../com/facebook/presto/ml/TestMLQueries.java | 2 +- .../presto/mongodb/MongoMetadata.java | 2 +- .../presto/mongodb/MongoPageSink.java | 202 ++- .../facebook/presto/mongodb/MongoSession.java | 2 +- .../mysql/TestMySqlDistributedQueries.java | 20 +- .../mysql/TestMySqlIntegrationSmokeTest.java | 26 +- .../orc/metadata/DwrfMetadataReader.java | 4 +- .../presto/orc/reader/ListStreamReader.java | 17 +- .../presto/orc/reader/MapStreamReader.java | 28 +- .../presto/orc/reader/StructStreamReader.java | 11 +- .../orc/TestTupleDomainOrcPredicate.java | 2 - .../com/facebook/presto/sql/parser/SqlBase.g4 | 6 +- .../presto/sql/ExpressionFormatter.java | 7 - .../com/facebook/presto/sql/SqlFormatter.java | 3 +- .../presto/sql/parser/AstBuilder.java | 23 +- .../presto/sql/parser/StatementSplitter.java | 4 +- .../facebook/presto/sql/tree/AddColumn.java | 2 +- .../facebook/presto/sql/tree/AstVisitor.java | 53 +- .../com/facebook/presto/sql/tree/Call.java | 2 +- .../com/facebook/presto/sql/tree/Cast.java | 2 +- .../com/facebook/presto/sql/tree/Commit.java | 2 +- .../facebook/presto/sql/tree/CreateTable.java | 2 +- .../facebook/presto/sql/tree/CreateView.java | 2 +- .../sql/tree/DataDefinitionStatement.java | 19 +- .../facebook/presto/sql/tree/Deallocate.java | 2 +- .../com/facebook/presto/sql/tree/Delete.java | 2 +- .../facebook/presto/sql/tree/DropTable.java | 2 +- .../facebook/presto/sql/tree/DropView.java | 2 +- .../sql/tree/ExpressionTreeRewriter.java | 5 +- .../facebook/presto/sql/tree/FrameBound.java | 17 +- .../com/facebook/presto/sql/tree/Grant.java | 2 +- .../sql/tree/LambdaArgumentDeclaration.java | 60 - .../presto/sql/tree/LambdaExpression.java | 10 +- .../com/facebook/presto/sql/tree/Prepare.java | 2 +- .../presto/sql/tree/RenameColumn.java | 2 +- .../facebook/presto/sql/tree/RenameTable.java | 2 +- .../presto/sql/tree/ResetSession.java | 2 +- .../com/facebook/presto/sql/tree/Revoke.java | 2 +- .../facebook/presto/sql/tree/Rollback.java | 2 +- .../facebook/presto/sql/tree/SetSession.java | 2 +- .../presto/sql/tree/StartTransaction.java | 2 +- .../presto/sql/parser/TestSqlParser.java | 82 +- .../sql/parser/TestStatementBuilder.java | 2 +- presto-product-tests/README.md | 60 +- presto-product-tests/bin/run_on_docker.sh | 17 +- .../conf/docker/common/cassandra.yml | 10 - .../conf/docker/multinode/compose.sh | 1 - .../singlenode-hdfs-impersonation/compose.sh | 1 - .../compose.sh | 1 - .../compose.sh | 1 - .../conf/docker/singlenode/compose.sh | 1 - .../presto/etc/catalog/cassandra.properties | 3 - .../conf/presto/etc/catalog/hive.properties | 1 - .../hive.properties | 1 - .../hive.properties | 1 - .../hive.properties | 1 - ...mpto-configuration-for-docker-default.yaml | 6 - ...pto-configuration-for-docker-kerberos.yaml | 9 +- presto-product-tests/pom.xml | 4 - .../com/facebook/presto/tests/TestGroups.java | 1 - .../CassandraTpchTableDefinitions.java | 48 - .../presto/tests/cassandra/Select.java | 59 - .../presto/tests/cli/PrestoCliTests.java | 50 +- .../hive/AllSimpleTypesTableDefinitions.java | 2 +- .../tests/hive/HiveTableDefinitions.java | 66 - .../TestAllDatatypesFromHiveConnector.java | 9 +- .../tests/hive/TestExternalHiveTable.java | 98 -- .../presto/tests/hive/TestHiveCoercion.java | 2 +- .../hive/TestTablePartitioningInsertInto.java | 77 +- .../QueryStatsClientModuleProvider.java | 8 +- .../presto/tests/utils/QueryExecutors.java | 5 - .../testcases/catalog/describe.result | 8 +- .../testcases/catalog/showColumns.result | 10 +- .../connectors/mysql/describe_real_table.sql | 10 +- .../connectors/mysql/describe_table.sql | 16 +- .../postgresql/describe_real_table.sql | 10 +- .../connectors/postgresql/describe_table.sql | 16 +- .../sql-tests/testcases/hive_tpch/q15.result | 2 +- .../sql-tests/testcases/hive_tpch/q15.sql | 2 +- .../sql-tests/testcases/hive_tpch/q6.result | 2 +- .../sql-tests/testcases/hive_tpch/q6.sql | 4 +- .../main/resources/tempto-configuration.yaml | 26 +- .../presto/raptor/RaptorBucketFunction.java | 56 +- .../presto/raptor/RaptorMetadata.java | 7 +- .../RaptorNodePartitioningProvider.java | 2 +- .../presto/raptor/RaptorPageSink.java | 18 +- .../raptor/metadata/DatabaseShardManager.java | 22 +- .../presto/raptor/metadata/SchemaDao.java | 3 +- .../raptor/storage/ShardRecoveryManager.java | 4 +- .../presto/raptor/util/DatabaseUtil.java | 28 +- .../raptor/RaptorBenchmarkQueryRunner.java | 2 +- .../raptor/TestRaptorBucketFunction.java | 92 -- .../TestRaptorIntegrationSmokeTest.java | 15 - .../metadata/TestDatabaseShardManager.java | 46 - .../raptor/metadata/TestRaptorMetadata.java | 3 +- .../raptor/storage/TestOrcStorageManager.java | 12 +- .../presto/decoder/csv/CsvFieldDecoder.java | 2 +- .../decoder/dummy/DummyFieldDecoder.java | 2 +- .../json/CustomDateTimeJsonFieldDecoder.java | 2 +- .../decoder/json/ISO8601JsonFieldDecoder.java | 2 +- .../presto/decoder/json/JsonFieldDecoder.java | 2 +- ...illisecondsSinceEpochJsonFieldDecoder.java | 2 +- .../decoder/json/RFC2822JsonFieldDecoder.java | 2 +- .../SecondsSinceEpochJsonFieldDecoder.java | 2 +- .../presto/decoder/raw/RawFieldDecoder.java | 2 +- .../facebook/presto/redis/RedisMetadata.java | 2 +- .../redis/RedisTableDescriptionSupplier.java | 4 +- .../decoder/hash/HashRedisFieldDecoder.java | 2 +- .../presto/redis/util/CodecSupplier.java | 3 +- .../AbstractResourceConfigurationManager.java | 27 +- .../ResourceGroupIdTemplate.java | 6 - .../db/ResourceGroupSpecBuilder.java | 2 +- ...FileResourceGroupConfigurationManager.java | 6 - .../resource_groups_config_bad_selector.json | 46 - .../facebook/presto/spi/ColumnMetadata.java | 18 +- .../presto/spi/ConnectorPageSink.java | 7 - .../presto/spi/ConnectorPageSource.java | 13 - .../presto/spi/DiscretePredicates.java | 9 +- .../presto/spi/block/AbstractArrayBlock.java | 123 +- .../facebook/presto/spi/block/ArrayBlock.java | 60 +- .../presto/spi/block/ArrayBlockBuilder.java | 85 +- .../presto/spi/block/ArrayBlockEncoding.java | 23 +- .../presto/spi/predicate/SortedRangeSet.java | 2 +- .../presto/spi/type/AbstractType.java | 22 +- .../facebook/presto/spi/type/CharType.java | 8 +- .../com/facebook/presto/spi/type/Chars.java | 38 - .../facebook/presto/spi/type/Decimals.java | 7 +- .../presto/spi/predicate/TestTupleDomain.java | 34 +- .../presto/spi/type/TestTypeSignature.java | 6 +- .../presto/spi/type/TestingTypeManager.java | 2 +- presto-tests/pom.xml | 19 - .../tests/AbstractTestDistributedQueries.java | 6 +- .../AbstractTestIntegrationSmokeTest.java | 40 +- .../presto/tests/AbstractTestQueries.java | 187 +-- .../presto/tests/DistributedQueryRunner.java | 2 +- .../presto/tests/StandaloneQueryRunner.java | 2 +- .../presto/tests/tpch/TpchIndexSpec.java | 2 +- .../tests/TestLocalBinarySpilledQueries.java | 62 - .../presto/tests/TestLocalQueries.java | 2 +- .../presto/tests/TestLocalQueriesIndexed.java | 2 +- .../facebook/presto/tpch/TpchMetadata.java | 6 +- .../facebook/presto/verifier/Validator.java | 16 +- 603 files changed, 5011 insertions(+), 14195 deletions(-) delete mode 100644 presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHolePageSource.java create mode 100644 presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/DelayPageSource.java delete mode 100644 presto-docs/src/main/sphinx/functions/lambda.rst delete mode 100644 presto-docs/src/main/sphinx/release/release-0.157.1.rst delete mode 100644 presto-docs/src/main/sphinx/release/release-0.158.rst delete mode 100644 presto-docs/src/main/sphinx/release/release-0.159.rst delete mode 100644 presto-docs/src/main/sphinx/release/release-0.160.rst delete mode 100644 presto-docs/src/main/sphinx/release/release-0.161.rst create mode 100644 presto-hive/src/main/java/org/apache/hadoop/security/UserGroupInformationShim.java delete mode 100644 presto-hive/src/test/java/com/facebook/presto/hive/TestPartitionOfflineException.java delete mode 100644 presto-hive/src/test/java/com/facebook/presto/hive/TestTableOfflineException.java delete mode 100644 presto-jmx/src/test/java/com/facebook/presto/connector/jmx/TestJmxConnectorConfig.java delete mode 100644 presto-main/src/main/java/com/facebook/presto/execution/FutureStateChange.java delete mode 100644 presto-main/src/main/java/com/facebook/presto/operator/aggregation/builder/MergingHashAggregationBuilder.java delete mode 100644 presto-main/src/main/java/com/facebook/presto/operator/aggregation/builder/SpillableHashAggregationBuilder.java delete mode 100644 presto-main/src/main/java/com/facebook/presto/operator/scalar/ApplyFunction.java delete mode 100644 presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayFilterFunction.java delete mode 100644 presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayReduceFunction.java delete mode 100644 presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayTransformFunction.java delete mode 100644 presto-main/src/main/java/com/facebook/presto/operator/scalar/EmptyMapConstructor.java delete mode 100644 presto-main/src/main/java/com/facebook/presto/operator/scalar/MapFilterFunction.java create mode 100644 presto-main/src/main/java/com/facebook/presto/server/ExecuteResource.java rename presto-product-tests/src/main/java/com/facebook/presto/tests/cassandra/TestConstants.java => presto-main/src/main/java/com/facebook/presto/server/ForExecute.java (54%) delete mode 100644 presto-main/src/main/java/com/facebook/presto/sql/analyzer/TypeSignatureProvider.java delete mode 100644 presto-main/src/main/java/com/facebook/presto/sql/gen/LambdaAndTryExpressionExtractor.java delete mode 100644 presto-main/src/main/java/com/facebook/presto/sql/gen/LambdaBytecodeGenerator.java delete mode 100644 presto-main/src/main/java/com/facebook/presto/sql/gen/ParameterAndType.java delete mode 100644 presto-main/src/main/java/com/facebook/presto/sql/gen/PreGeneratedExpressions.java create mode 100644 presto-main/src/main/java/com/facebook/presto/sql/gen/TryExpressionExtractor.java delete mode 100644 presto-main/src/main/java/com/facebook/presto/sql/gen/VarArgsToMapAdapterGenerator.java create mode 100644 presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/EvaluateConstantApply.java rename presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/{RemoveUnreferencedScalarInputApplyNodes.java => RemoveRedundantApply.java} (58%) delete mode 100644 presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TransformExistsApplyToScalarApply.java delete mode 100644 presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TransformQuantifiedComparisonApplyToScalarApply.java delete mode 100644 presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/VerifyOnlyOneOutputNode.java delete mode 100644 presto-main/src/main/java/com/facebook/presto/sql/relational/LambdaDefinitionExpression.java delete mode 100644 presto-main/src/main/java/com/facebook/presto/sql/relational/VariableReferenceExpression.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayFilter.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkMapConcat.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/operator/scalar/TestApplyFunction.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/operator/scalar/TestArrayFilterFunction.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/operator/scalar/TestArrayReduceFunction.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/operator/scalar/TestArrayTransformFunction.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/operator/scalar/TestMapFilterFunction.java create mode 100644 presto-main/src/test/java/com/facebook/presto/server/TestExecuteResource.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/server/remotetask/TestHttpRemoteTask.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/TestPlanMatchingFramework.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/TestPredicatePushdown.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/TestQuantifiedComparison.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/AggregationFunctionMatcher.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/Alias.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/AnySymbol.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/AnySymbolReference.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/BasePlanTest.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ColumnReference.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/EquiJoinClauseProvider.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ExpectedValueProvider.java create mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ExpressionAliases.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ExpressionMatcher.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/FunctionCallProvider.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/MatchResult.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/OutputMatcher.java rename presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/{SymbolAlias.java => PlanMatchingContext.java} (50%) delete mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanTestSymbol.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/RvalueMatcher.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/SpecificationProvider.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/SymbolAliases.java create mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/SymbolMatcher.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ValuesMatcher.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/WindowFunctionMatcher.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestSetFlatteningOptimizer.java delete mode 100644 presto-main/src/test/java/com/facebook/presto/sql/planner/sanity/TestVerifyOnlyOneOutputNode.java rename presto-main/src/main/java/com/facebook/presto/spiller/SpillerFactoryWithStats.java => presto-parser/src/main/java/com/facebook/presto/sql/tree/DataDefinitionStatement.java (61%) delete mode 100644 presto-parser/src/main/java/com/facebook/presto/sql/tree/LambdaArgumentDeclaration.java delete mode 100644 presto-product-tests/conf/docker/common/cassandra.yml delete mode 100644 presto-product-tests/conf/presto/etc/catalog/cassandra.properties delete mode 100644 presto-product-tests/src/main/java/com/facebook/presto/tests/cassandra/CassandraTpchTableDefinitions.java delete mode 100644 presto-product-tests/src/main/java/com/facebook/presto/tests/cassandra/Select.java delete mode 100644 presto-product-tests/src/main/java/com/facebook/presto/tests/hive/HiveTableDefinitions.java delete mode 100644 presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestExternalHiveTable.java delete mode 100644 presto-raptor/src/test/java/com/facebook/presto/raptor/TestRaptorBucketFunction.java delete mode 100644 presto-resource-group-managers/src/test/resources/resource_groups_config_bad_selector.json delete mode 100644 presto-tests/src/test/java/com/facebook/presto/tests/TestLocalBinarySpilledQueries.java diff --git a/.travis.yml b/.travis.yml index 49c52e08ef25..ebbc2e1bb159 100644 --- a/.travis.yml +++ b/.travis.yml @@ -5,7 +5,6 @@ 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} matrix: - MAVEN_CHECKS=true - TEST_SPECIFIC_MODULES=presto-tests @@ -76,6 +75,11 @@ script: if [[ -v HIVE_TESTS ]]; then presto-hive-hadoop2/bin/run_on_docker.sh fi + - | + # Build presto-server-rpm for later artifact upload + if [[ -v DEPLOY_S3_ACCESS_KEY && -v PRODUCT_TESTS ]]; then + ./mvnw install $MAVEN_FAST_INSTALL -pl presto-server-rpm + fi before_cache: # Make the cache stable between builds by removing build output @@ -100,35 +104,12 @@ before_deploy: deploy: on: all_branches: true - condition: -v DEPLOY_S3_ACCESS_KEY && -v MAVEN_CHECKS + condition: -v DEPLOY_S3_ACCESS_KEY && -v PRODUCT_TESTS provider: s3 access_key_id: ${DEPLOY_S3_ACCESS_KEY} secret_access_key: ${DEPLOY_S3_SECRET_KEY} bucket: ${DEPLOY_S3_BUCKET} skip_cleanup: true local-dir: /tmp/artifacts - upload-dir: ${ARTIFACTS_UPLOAD_PATH} + upload-dir: travis_build_artifacts/${TRAVIS_REPO_SLUG}/${TRAVIS_BRANCH}/${TRAVIS_JOB_NUMBER} acl: public_read - -after_script: -- | - if [[ -v DEPLOY_S3_ACCESS_KEY ]]; then - sudo pip install awscli - export AWS_ACCESS_KEY_ID=${DEPLOY_S3_ACCESS_KEY} - export AWS_SECRET_ACCESS_KEY=${DEPLOY_S3_SECRET_KEY} - - JOB_ARTIFACTS_URL_PREFIX=s3://${DEPLOY_S3_BUCKET}/${ARTIFACTS_UPLOAD_PATH}/travis_jobs/${TRAVIS_JOB_NUMBER}-run - JOB_RUN_ATTEMPTS=$( aws s3 ls ${JOB_ARTIFACTS_URL_PREFIX} | wc -l | tr -d '[:space:]' ) - JOB_STATUS=$( [ "$TRAVIS_TEST_RESULT" == "0" ] && echo SUCCESS || echo FAILURE ) - - mkdir -p /tmp/job_artifacts/ - rsync -av -m \ - --include='**/' \ - --include='**/surefire-reports/**.xml' \ - --include='**/surefire-reports/emailable-report.html' \ - --exclude='*' \ - . /tmp/job_artifacts/ - wget https://api.travis-ci.org/jobs/${TRAVIS_JOB_ID}/log.txt?deansi=true -O /tmp/job_artifacts/log.txt - - aws s3 sync /tmp/job_artifacts ${JOB_ARTIFACTS_URL_PREFIX}_$((JOB_RUN_ATTEMPTS + 1))-${JOB_STATUS} - fi diff --git a/README.md b/README.md index 048c247eff51..a60882435bcc 100644 --- a/README.md +++ b/README.md @@ -38,7 +38,7 @@ After opening the project in IntelliJ, double check that the Java SDK is properl Presto comes with sample configuration that should work out-of-the-box for development. Use the following options to create a run configuration: * Main Class: `com.facebook.presto.server.PrestoServer` -* VM Options: `-ea -XX:+UseG1GC -XX:G1HeapRegionSize=32M -XX:+UseGCOverheadLimit -XX:+ExplicitGCInvokesConcurrent -Xmx2G -Dconfig=etc/config.properties -Dlog.levels-file=etc/log.properties` +* VM Options: `-ea -Xmx2G -Dconfig=etc/config.properties -Dlog.levels-file=etc/log.properties` * Working directory: `$MODULE_DIR$` * Use classpath of module: `presto-main` diff --git a/pom.xml b/pom.xml index a321d331717c..4544307a43fa 100644 --- a/pom.xml +++ b/pom.xml @@ -50,7 +50,7 @@ ${dep.airlift.version} 0.27 1.11.30 - 1.18 + 1.14 diff --git a/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/MetadataUtil.java b/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/MetadataUtil.java index 82d6c3fc7d72..bce7ffb3b5f7 100644 --- a/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/MetadataUtil.java +++ b/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/MetadataUtil.java @@ -16,6 +16,7 @@ import com.facebook.presto.spi.type.StandardTypes; import com.facebook.presto.spi.type.Type; import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; import com.fasterxml.jackson.databind.deser.std.FromStringDeserializer; import com.google.common.collect.ImmutableMap; import io.airlift.json.JsonCodec; @@ -40,7 +41,7 @@ private MetadataUtil() {} static { ObjectMapperProvider provider = new ObjectMapperProvider(); - provider.setJsonDeserializers(ImmutableMap.of(Type.class, new TestingTypeDeserializer())); + provider.setJsonDeserializers(ImmutableMap., JsonDeserializer>of(Type.class, new TestingTypeDeserializer())); JsonCodecFactory codecFactory = new JsonCodecFactory(provider); COLUMN_CODEC = codecFactory.jsonCodec(JdbcColumnHandle.class); TABLE_CODEC = codecFactory.jsonCodec(JdbcTableHandle.class); @@ -50,7 +51,7 @@ private MetadataUtil() {} public static final class TestingTypeDeserializer extends FromStringDeserializer { - private final Map types = ImmutableMap.of( + private final Map types = ImmutableMap.of( StandardTypes.BIGINT, BIGINT, StandardTypes.VARCHAR, VARCHAR); diff --git a/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/TestJdbcOutputTableHandle.java b/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/TestJdbcOutputTableHandle.java index c5ece8840c11..83fba31171fe 100644 --- a/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/TestJdbcOutputTableHandle.java +++ b/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/TestJdbcOutputTableHandle.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.plugin.jdbc; +import com.facebook.presto.spi.type.Type; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.testng.annotations.Test; @@ -32,7 +33,7 @@ public void testJsonRoundTrip() "schema", "table", ImmutableList.of("abc", "xyz"), - ImmutableList.of(VARCHAR, VARCHAR), + ImmutableList.of(VARCHAR, VARCHAR), "tmp_table", "jdbc:junk", ImmutableMap.of("user", "test")); diff --git a/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/TestJdbcRecordSet.java b/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/TestJdbcRecordSet.java index 7360b21be940..4397f6c5d434 100644 --- a/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/TestJdbcRecordSet.java +++ b/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/TestJdbcRecordSet.java @@ -76,7 +76,7 @@ public void testGetColumnTypes() new JdbcColumnHandle("test", "text", VARCHAR))); assertEquals(recordSet.getColumnTypes(), ImmutableList.of(BIGINT, BIGINT, VARCHAR)); - recordSet = new JdbcRecordSet(jdbcClient, split, ImmutableList.of()); + recordSet = new JdbcRecordSet(jdbcClient, split, ImmutableList.of()); assertEquals(recordSet.getColumnTypes(), ImmutableList.of()); } diff --git a/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/TestJdbcRecordSetProvider.java b/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/TestJdbcRecordSetProvider.java index cf2a7d771488..43209bf86491 100644 --- a/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/TestJdbcRecordSetProvider.java +++ b/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/TestJdbcRecordSetProvider.java @@ -111,52 +111,52 @@ public void testTupleDomain() { // single value getCursor(table, ImmutableList.of(textColumn, valueColumn), TupleDomain.withColumnDomains( - ImmutableMap.of(textColumn, Domain.singleValue(VARCHAR, utf8Slice("foo"))) + ImmutableMap.of(textColumn, Domain.singleValue(VARCHAR, utf8Slice("foo"))) )); // multiple values (string) getCursor(table, ImmutableList.of(textColumn, valueColumn), TupleDomain.withColumnDomains( - ImmutableMap.of(textColumn, Domain.union(ImmutableList.of(Domain.singleValue(VARCHAR, utf8Slice("foo")), Domain.singleValue(VARCHAR, utf8Slice("bar"))))) + ImmutableMap.of(textColumn, Domain.union(ImmutableList.of(Domain.singleValue(VARCHAR, utf8Slice("foo")), Domain.singleValue(VARCHAR, utf8Slice("bar"))))) )); // inequality (string) getCursor(table, ImmutableList.of(textColumn, valueColumn), TupleDomain.withColumnDomains( - ImmutableMap.of(textColumn, Domain.create(ValueSet.ofRanges(Range.greaterThan(VARCHAR, utf8Slice("foo"))), false)) + ImmutableMap.of(textColumn, Domain.create(ValueSet.ofRanges(Range.greaterThan(VARCHAR, utf8Slice("foo"))), false)) )); getCursor(table, ImmutableList.of(textColumn, valueColumn), TupleDomain.withColumnDomains( - ImmutableMap.of(textColumn, Domain.create(ValueSet.ofRanges(Range.greaterThan(VARCHAR, utf8Slice("foo"))), false)) + ImmutableMap.of(textColumn, Domain.create(ValueSet.ofRanges(Range.greaterThan(VARCHAR, utf8Slice("foo"))), false)) )); getCursor(table, ImmutableList.of(textColumn, valueColumn), TupleDomain.withColumnDomains( - ImmutableMap.of(textColumn, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(VARCHAR, utf8Slice("foo"))), false)) + ImmutableMap.of(textColumn, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(VARCHAR, utf8Slice("foo"))), false)) )); getCursor(table, ImmutableList.of(textColumn, valueColumn), TupleDomain.withColumnDomains( - ImmutableMap.of(textColumn, Domain.create(ValueSet.ofRanges(Range.lessThan(VARCHAR, utf8Slice("foo"))), false)) + ImmutableMap.of(textColumn, Domain.create(ValueSet.ofRanges(Range.lessThan(VARCHAR, utf8Slice("foo"))), false)) )); // is null getCursor(table, ImmutableList.of(textColumn, valueColumn), TupleDomain.withColumnDomains( - ImmutableMap.of(textColumn, Domain.onlyNull(VARCHAR)) + ImmutableMap.of(textColumn, Domain.onlyNull(VARCHAR)) )); // not null getCursor(table, ImmutableList.of(textColumn, valueColumn), TupleDomain.withColumnDomains( - ImmutableMap.of(textColumn, Domain.notNull(VARCHAR)) + ImmutableMap.of(textColumn, Domain.notNull(VARCHAR)) )); // specific value or null getCursor(table, ImmutableList.of(textColumn, valueColumn), TupleDomain.withColumnDomains( - ImmutableMap.of(textColumn, Domain.union(ImmutableList.of(Domain.singleValue(VARCHAR, utf8Slice("foo")), Domain.onlyNull(VARCHAR)))) + ImmutableMap.of(textColumn, Domain.union(ImmutableList.of(Domain.singleValue(VARCHAR, utf8Slice("foo")), Domain.onlyNull(VARCHAR)))) )); getCursor(table, ImmutableList.of(textColumn, valueColumn), TupleDomain.withColumnDomains( - ImmutableMap.of(textColumn, Domain.create(ValueSet.ofRanges(Range.range(VARCHAR, utf8Slice("bar"), true, utf8Slice("foo"), true)), false)) + ImmutableMap.of(textColumn, Domain.create(ValueSet.ofRanges(Range.range(VARCHAR, utf8Slice("bar"), true, utf8Slice("foo"), true)), false)) )); getCursor(table, ImmutableList.of(textColumn, textShortColumn, valueColumn), TupleDomain.withColumnDomains( - ImmutableMap.of( + ImmutableMap.of( textColumn, Domain.create(ValueSet.ofRanges( Range.range(VARCHAR, utf8Slice("bar"), true, utf8Slice("foo"), true), @@ -174,7 +174,7 @@ public void testTupleDomain() )); getCursor(table, ImmutableList.of(textColumn, valueColumn), TupleDomain.withColumnDomains( - ImmutableMap.of( + ImmutableMap.of( textColumn, Domain.create(ValueSet.ofRanges( Range.range(VARCHAR, utf8Slice("bar"), true, utf8Slice("foo"), true), diff --git a/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/TestJdbcSplit.java b/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/TestJdbcSplit.java index 808339b6b9c7..8264ae7565ca 100644 --- a/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/TestJdbcSplit.java +++ b/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/TestJdbcSplit.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.plugin.jdbc; +import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.predicate.TupleDomain; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -24,7 +25,7 @@ public class TestJdbcSplit { - private final JdbcSplit split = new JdbcSplit("connectorId", "catalog", "schemaName", "tableName", "connectionUrl", ImmutableMap.of(), TupleDomain.all()); + private final JdbcSplit split = new JdbcSplit("connectorId", "catalog", "schemaName", "tableName", "connectionUrl", ImmutableMap.of(), TupleDomain.all()); @Test public void testAddresses() @@ -33,7 +34,7 @@ public void testAddresses() assertEquals(split.getAddresses(), ImmutableList.of()); assertEquals(split.isRemotelyAccessible(), true); - JdbcSplit jdbcSplit = new JdbcSplit("connectorId", "catalog", "schemaName", "tableName", "connectionUrl", ImmutableMap.of(), TupleDomain.all()); + JdbcSplit jdbcSplit = new JdbcSplit("connectorId", "catalog", "schemaName", "tableName", "connectionUrl", ImmutableMap.of(), TupleDomain.all()); assertEquals(jdbcSplit.getAddresses(), ImmutableList.of()); } diff --git a/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/TestingDatabase.java b/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/TestingDatabase.java index 23688d6340db..192f84234b69 100644 --- a/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/TestingDatabase.java +++ b/presto-base-jdbc/src/test/java/com/facebook/presto/plugin/jdbc/TestingDatabase.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.plugin.jdbc; +import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.ConnectorSplitSource; import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.spi.predicate.TupleDomain; @@ -93,7 +94,7 @@ public JdbcSplit getSplit(String schemaName, String tableName) throws InterruptedException { JdbcTableHandle jdbcTableHandle = jdbcClient.getTableHandle(new SchemaTableName(schemaName, tableName)); - JdbcTableLayoutHandle jdbcLayoutHandle = new JdbcTableLayoutHandle(jdbcTableHandle, TupleDomain.all()); + JdbcTableLayoutHandle jdbcLayoutHandle = new JdbcTableLayoutHandle(jdbcTableHandle, TupleDomain.all()); ConnectorSplitSource splits = jdbcClient.getSplits(jdbcLayoutHandle); return (JdbcSplit) getOnlyElement(getFutureValue(splits.getNextBatch(1000))); } diff --git a/presto-benchmark-driver/src/main/java/com/facebook/presto/benchmark/driver/BenchmarkSchema.java b/presto-benchmark-driver/src/main/java/com/facebook/presto/benchmark/driver/BenchmarkSchema.java index 6502c3eaaec5..d64bca9ce929 100644 --- a/presto-benchmark-driver/src/main/java/com/facebook/presto/benchmark/driver/BenchmarkSchema.java +++ b/presto-benchmark-driver/src/main/java/com/facebook/presto/benchmark/driver/BenchmarkSchema.java @@ -26,7 +26,7 @@ public class BenchmarkSchema public BenchmarkSchema(String name) { - this(name, ImmutableMap.of()); + this(name, ImmutableMap.of()); } public BenchmarkSchema(String name, Map tags) 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 0a3c9cd30486..6dc582ae2c8d 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 @@ -44,7 +44,7 @@ public class Suite public Suite(String name, Map sessionProperties, Iterable schemaNameTemplates, Iterable queryNamePatterns) { this.name = requireNonNull(name, "name is null"); - this.sessionProperties = sessionProperties == null ? ImmutableMap.of() : ImmutableMap.copyOf(sessionProperties); + this.sessionProperties = sessionProperties == null ? ImmutableMap.of() : ImmutableMap.copyOf(sessionProperties); this.schemaNameTemplates = ImmutableList.copyOf(requireNonNull(schemaNameTemplates, "schemaNameTemplates is null")); this.queryNamePatterns = ImmutableList.copyOf(requireNonNull(queryNamePatterns, "queryNamePatterns is null")); } diff --git a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/AbstractOperatorBenchmark.java b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/AbstractOperatorBenchmark.java index e15632ef07fa..2932cbe0a424 100644 --- a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/AbstractOperatorBenchmark.java +++ b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/AbstractOperatorBenchmark.java @@ -95,7 +95,7 @@ protected OperatorFactory createTableScanOperator(int operatorId, PlanNodeId pla protected OperatorFactory createHashProjectOperator(int operatorId, PlanNodeId planNodeId, List types) { - return LocalQueryRunner.createHashProjectOperator(operatorId, planNodeId, types); + return localQueryRunner.createHashProjectOperator(operatorId, planNodeId, types); } protected abstract List createDrivers(TaskContext taskContext); diff --git a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/BenchmarkQueryRunner.java b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/BenchmarkQueryRunner.java index 037c481a83be..09fc92f0a30a 100644 --- a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/BenchmarkQueryRunner.java +++ b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/BenchmarkQueryRunner.java @@ -51,7 +51,7 @@ public static LocalQueryRunner createLocalQueryRunner(boolean hashingEnabled) LocalQueryRunner localQueryRunner = new LocalQueryRunner(session); // add tpch - localQueryRunner.createCatalog("tpch", new TpchConnectorFactory(1), ImmutableMap.of()); + localQueryRunner.createCatalog("tpch", new TpchConnectorFactory(1), ImmutableMap.of()); return localQueryRunner; } diff --git a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/BenchmarkSuite.java b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/BenchmarkSuite.java index dba1ddb7b885..bd90e5bd527f 100644 --- a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/BenchmarkSuite.java +++ b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/BenchmarkSuite.java @@ -38,7 +38,7 @@ public static List createBenchmarks(LocalQueryRunner localQue Session optimizeHashSession = Session.builder(localQueryRunner.getDefaultSession()) .setSystemProperty(OPTIMIZE_HASH_GENERATION, "true") .build(); - return ImmutableList.of( + return ImmutableList.of( // hand built benchmarks new CountAggregationBenchmark(localQueryRunner), new DoubleSumAggregationBenchmark(localQueryRunner), diff --git a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HandTpchQuery1.java b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HandTpchQuery1.java index 958a3308fe94..3c9cc464b2fb 100644 --- a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HandTpchQuery1.java +++ b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HandTpchQuery1.java @@ -133,7 +133,7 @@ protected List createOperatorFactories() public static class TpchQuery1Operator implements com.facebook.presto.operator.Operator // TODO: use import when Java 7 compiler bug is fixed { - private static final ImmutableList TYPES = ImmutableList.of( + private static final ImmutableList TYPES = ImmutableList.of( VARCHAR, VARCHAR, DOUBLE, diff --git a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HandTpchQuery6.java b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HandTpchQuery6.java index 70672422b8b0..ca95bbfd69fa 100644 --- a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HandTpchQuery6.java +++ b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HandTpchQuery6.java @@ -64,7 +64,7 @@ protected List createOperatorFactories() // and quantity < 24; OperatorFactory tableScanOperator = createTableScanOperator(0, new PlanNodeId("test"), "lineitem", "extendedprice", "discount", "shipdate", "quantity"); - FilterAndProjectOperator.FilterAndProjectOperatorFactory tpchQuery6Operator = new FilterAndProjectOperator.FilterAndProjectOperatorFactory(1, new PlanNodeId("test"), () -> new TpchQuery6Processor(), ImmutableList.of(DOUBLE)); + FilterAndProjectOperator.FilterAndProjectOperatorFactory tpchQuery6Operator = new FilterAndProjectOperator.FilterAndProjectOperatorFactory(1, new PlanNodeId("test"), () -> new TpchQuery6Processor(), ImmutableList.of(DOUBLE)); AggregationOperatorFactory aggregationOperator = new AggregationOperatorFactory( 2, diff --git a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HashBuildAndJoinBenchmark.java b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HashBuildAndJoinBenchmark.java index 3c96b46325dd..4b19aae5628d 100644 --- a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HashBuildAndJoinBenchmark.java +++ b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/HashBuildAndJoinBenchmark.java @@ -21,6 +21,7 @@ import com.facebook.presto.operator.LookupJoinOperators; import com.facebook.presto.operator.OperatorFactory; import com.facebook.presto.operator.TaskContext; +import com.facebook.presto.spi.type.Type; import com.facebook.presto.sql.planner.plan.PlanNodeId; import com.facebook.presto.testing.LocalQueryRunner; import com.facebook.presto.testing.NullOutputOperator.NullOutputOperatorFactory; @@ -68,7 +69,7 @@ protected List createDrivers(TaskContext taskContext) OperatorFactory source = ordersTableScan; Optional hashChannel = Optional.empty(); if (hashEnabled) { - source = createHashProjectOperator(1, new PlanNodeId("test"), ImmutableList.of(BIGINT, DOUBLE)); + source = createHashProjectOperator(1, new PlanNodeId("test"), ImmutableList.of(BIGINT, DOUBLE)); driversBuilder.add(source); hashChannel = Optional.of(2); } @@ -86,7 +87,7 @@ protected List createDrivers(TaskContext taskContext) source = lineItemTableScan; hashChannel = Optional.empty(); if (hashEnabled) { - source = createHashProjectOperator(1, new PlanNodeId("test"), ImmutableList.of(BIGINT, BIGINT)); + source = createHashProjectOperator(1, new PlanNodeId("test"), ImmutableList.of(BIGINT, BIGINT)); joinDriversBuilder.add(source); hashChannel = Optional.of(2); } diff --git a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/PredicateFilterBenchmark.java b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/PredicateFilterBenchmark.java index 5e9bc7be198d..96b48ad50a40 100644 --- a/presto-benchmark/src/main/java/com/facebook/presto/benchmark/PredicateFilterBenchmark.java +++ b/presto-benchmark/src/main/java/com/facebook/presto/benchmark/PredicateFilterBenchmark.java @@ -19,6 +19,7 @@ import com.facebook.presto.operator.OperatorFactory; import com.facebook.presto.spi.RecordCursor; import com.facebook.presto.spi.block.Block; +import com.facebook.presto.spi.type.Type; import com.facebook.presto.sql.planner.plan.PlanNodeId; import com.facebook.presto.testing.LocalQueryRunner; import com.google.common.collect.ImmutableList; @@ -47,7 +48,7 @@ protected List createOperatorFactories() 1, new PlanNodeId("test"), () -> new GenericPageProcessor(new DoubleFilter(50000.00), ImmutableList.of(singleColumn(DOUBLE, 0))), - ImmutableList.of(DOUBLE)); + ImmutableList.of(DOUBLE)); return ImmutableList.of(tableScanOperator, filterAndProjectOperator); } diff --git a/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleConnectorFactory.java b/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleConnectorFactory.java index f4be542b48d4..91a0bf8b8fae 100644 --- a/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleConnectorFactory.java +++ b/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleConnectorFactory.java @@ -48,7 +48,7 @@ public Connector create(String connectorId, Map requiredConfig, return new BlackHoleConnector( new BlackHoleMetadata(), new BlackHoleSplitManager(), - new BlackHolePageSourceProvider(executorService), + new BlackHolePageSourceProvider(), new BlackHolePageSinkProvider(executorService), new BlackHoleNodePartitioningProvider(context.getNodeManager()), context.getTypeManager(), diff --git a/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHolePageSink.java b/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHolePageSink.java index b7e09520afe9..eaa9695cf258 100644 --- a/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHolePageSink.java +++ b/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHolePageSink.java @@ -26,16 +26,14 @@ import static io.airlift.concurrent.MoreFutures.toCompletableFuture; import static java.util.Objects.requireNonNull; +import static java.util.concurrent.CompletableFuture.completedFuture; import static java.util.concurrent.TimeUnit.MILLISECONDS; class BlackHolePageSink implements ConnectorPageSink { - private static final CompletableFuture> NON_BLOCKED = CompletableFuture.completedFuture(ImmutableList.of()); - private final ListeningScheduledExecutorService executorService; private final long pageProcessingDelayMillis; - private CompletableFuture> appendFuture = NON_BLOCKED; public BlackHolePageSink(ListeningScheduledExecutorService executorService, Duration pageProcessingDelay) { @@ -45,23 +43,17 @@ public BlackHolePageSink(ListeningScheduledExecutorService executorService, Dura @Override public CompletableFuture appendPage(Page page) - { - appendFuture = scheduleAppend(); - return appendFuture; - } - - private CompletableFuture> scheduleAppend() { if (pageProcessingDelayMillis > 0) { - return toCompletableFuture(executorService.schedule(() -> ImmutableList.of(), pageProcessingDelayMillis, MILLISECONDS)); + return toCompletableFuture(executorService.schedule(() -> null, pageProcessingDelayMillis, MILLISECONDS)); } - return NON_BLOCKED; + return NOT_BLOCKED; } @Override public CompletableFuture> finish() { - return appendFuture; + return completedFuture(ImmutableList.of()); } @Override diff --git a/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHolePageSource.java b/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHolePageSource.java deleted file mode 100644 index d5b94b524cb5..000000000000 --- a/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHolePageSource.java +++ /dev/null @@ -1,125 +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.plugin.blackhole; - -import com.facebook.presto.spi.ConnectorPageSource; -import com.facebook.presto.spi.Page; -import com.google.common.util.concurrent.ListeningScheduledExecutorService; -import io.airlift.units.Duration; - -import java.io.IOException; -import java.util.concurrent.CompletableFuture; - -import static com.google.common.base.Preconditions.checkArgument; -import static io.airlift.concurrent.MoreFutures.getFutureValue; -import static io.airlift.concurrent.MoreFutures.toCompletableFuture; -import static java.util.Objects.requireNonNull; -import static java.util.concurrent.TimeUnit.MILLISECONDS; - -class BlackHolePageSource - implements ConnectorPageSource -{ - private final Page page; - private int pagesLeft; - private final ListeningScheduledExecutorService executorService; - private final long pageProcessingDelayInMillis; - private final long totalBytes; - private long completedBytes; - private final long memoryUsageBytes; - private boolean closed; - private CompletableFuture currentPage; - - BlackHolePageSource(Page page, int count, ListeningScheduledExecutorService executorService, Duration pageProcessingDelay) - { - this.page = requireNonNull(page, "page is null"); - checkArgument(count >= 0, "count is negative"); - this.pagesLeft = count; - this.executorService = requireNonNull(executorService, "executorService is null"); - this.pageProcessingDelayInMillis = requireNonNull(pageProcessingDelay, "pageProcessingDelay is null").toMillis(); - this.totalBytes = page.getSizeInBytes() * count; - this.memoryUsageBytes = page.getSizeInBytes(); - } - - @Override - public Page getNextPage() - { - if (isFinished()) { - return null; - } - - if (currentPage != null) { - Page page = getFutureValue(currentPage); - currentPage = null; - return page; - } - - pagesLeft--; - completedBytes += page.getSizeInBytes(); - - if (pageProcessingDelayInMillis == 0) { - return page; - } - else { - currentPage = toCompletableFuture(executorService.schedule(() -> page, pageProcessingDelayInMillis, MILLISECONDS)); - return null; - } - } - - @Override - public CompletableFuture isBlocked() - { - if (currentPage != null) { - return currentPage; - } - return NOT_BLOCKED; - } - - @Override - public boolean isFinished() - { - return closed || (pagesLeft == 0 && currentPage == null); - } - - @Override - public void close() - throws IOException - { - closed = true; - } - - @Override - public long getTotalBytes() - { - return totalBytes; - } - - @Override - public long getCompletedBytes() - { - return completedBytes; - } - - @Override - public long getReadTimeNanos() - { - return 0; - } - - @Override - public long getSystemMemoryUsage() - { - return memoryUsageBytes; - } -} diff --git a/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHolePageSourceProvider.java b/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHolePageSourceProvider.java index 00b73c6fe458..cf97597a54c7 100644 --- a/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHolePageSourceProvider.java +++ b/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHolePageSourceProvider.java @@ -29,7 +29,7 @@ import com.facebook.presto.spi.type.VarcharType; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; -import com.google.common.util.concurrent.ListeningScheduledExecutorService; +import com.google.common.collect.Iterables; import io.airlift.slice.Slice; import io.airlift.slice.Slices; @@ -58,13 +58,6 @@ public final class BlackHolePageSourceProvider implements ConnectorPageSourceProvider { - private final ListeningScheduledExecutorService executorService; - - public BlackHolePageSourceProvider(ListeningScheduledExecutorService executorService) - { - this.executorService = requireNonNull(executorService, "executorService is null"); - } - @Override public ConnectorPageSource createPageSource( ConnectorTransactionHandle transactionHandle, @@ -81,8 +74,11 @@ public ConnectorPageSource createPageSource( } List types = builder.build(); - Page page = generateZeroPage(types, blackHoleSplit.getRowsPerPage(), blackHoleSplit.getFieldsLength()); - return new BlackHolePageSource(page, blackHoleSplit.getPagesCount(), executorService, blackHoleSplit.getPageProcessingDelay()); + Iterable pages = Iterables.limit( + Iterables.cycle(generateZeroPage(types, blackHoleSplit.getRowsPerPage(), blackHoleSplit.getFieldsLength())), + blackHoleSplit.getPagesCount() + ); + return new DelayPageSource(pages, blackHoleSplit.getPageProcessingDelay()); } private Page generateZeroPage(List types, int rowsCount, int fieldLength) diff --git a/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleSplitManager.java b/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleSplitManager.java index 746378569f84..b342f20e7534 100644 --- a/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleSplitManager.java +++ b/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/BlackHoleSplitManager.java @@ -35,7 +35,7 @@ public ConnectorSplitSource getSplits(ConnectorTransactionHandle transactionHand BlackHoleTableLayoutHandle.class, "BlackHoleTableLayoutHandle"); - ImmutableList.Builder builder = ImmutableList.builder(); + ImmutableList.Builder builder = ImmutableList.builder(); for (int i = 0; i < layout.getSplitCount(); i++) { builder.add( diff --git a/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/DelayPageSource.java b/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/DelayPageSource.java new file mode 100644 index 000000000000..31fada155783 --- /dev/null +++ b/presto-blackhole/src/main/java/com/facebook/presto/plugin/blackhole/DelayPageSource.java @@ -0,0 +1,55 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.facebook.presto.plugin.blackhole; + +import com.facebook.presto.spi.FixedPageSource; +import com.facebook.presto.spi.Page; +import com.google.common.base.Throwables; +import io.airlift.units.Duration; + +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.MILLISECONDS; + +class DelayPageSource + extends FixedPageSource +{ + private final long pageProcessingDelayInMillis; + + public DelayPageSource(Iterable pages, Duration pageProcessingDelay) + { + super(pages); + this.pageProcessingDelayInMillis = requireNonNull(pageProcessingDelay, "pageProcessingDelay is null").toMillis(); + } + + @Override + public Page getNextPage() + { + if (isFinished()) { + return null; + } + + if (pageProcessingDelayInMillis > 0) { + try { + MILLISECONDS.sleep(pageProcessingDelayInMillis); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw Throwables.propagate(e); + } + } + + return super.getNextPage(); + } +} diff --git a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/ArrayOpCode.java b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/ArrayOpCode.java index f0191fa26654..a7028b24fa9b 100644 --- a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/ArrayOpCode.java +++ b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/ArrayOpCode.java @@ -106,7 +106,7 @@ public static ArrayOpCode getArrayOpCode(ParameterizedType type) static Map, ArrayOpCode> initializeArrayOpCodeMap() { - ImmutableMap.Builder, ArrayOpCode> builder = ImmutableMap.builder(); + ImmutableMap.Builder, ArrayOpCode> builder = ImmutableMap., ArrayOpCode>builder(); for (ArrayOpCode arrayOpCode : values()) { if (arrayOpCode.getType() != null) { builder.put(arrayOpCode.getType(), arrayOpCode); diff --git a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/ClassDefinition.java b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/ClassDefinition.java index 3ea1903c52ff..4ce42d26daea 100644 --- a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/ClassDefinition.java +++ b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/ClassDefinition.java @@ -73,7 +73,7 @@ public ClassDefinition( this.superClass = superClass; this.interfaces.addAll(ImmutableList.copyOf(interfaces)); - classInitializer = new MethodDefinition(this, a(STATIC), "", ParameterizedType.type(void.class), ImmutableList.of()); + classInitializer = new MethodDefinition(this, a(STATIC), "", ParameterizedType.type(void.class), ImmutableList.of()); } public Set getAccess() diff --git a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/ClassInfoLoader.java b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/ClassInfoLoader.java index fc809af0b2e7..e6913170077d 100644 --- a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/ClassInfoLoader.java +++ b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/ClassInfoLoader.java @@ -52,7 +52,7 @@ public static ClassInfoLoader createClassInfoLoader(Iterable cl classDefinition.visit(classNode); classNodes.put(classDefinition.getType(), classNode); } - return new ClassInfoLoader(classNodes.build(), ImmutableMap.of(), classLoader, true); + return new ClassInfoLoader(classNodes.build(), ImmutableMap.of(), classLoader, true); } private final Map classNodes; diff --git a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/Variable.java b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/Variable.java index 52c121653f89..b018016d978d 100644 --- a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/Variable.java +++ b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/Variable.java @@ -92,7 +92,7 @@ public BytecodeNode getBytecode(MethodGenerationContext generationContext) @Override public List getChildNodes() { - return ImmutableList.of(value); + return ImmutableList.of(value); } @Override diff --git a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/control/DoWhileLoop.java b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/control/DoWhileLoop.java index a5cd8f4048f2..1196df4c451c 100644 --- a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/control/DoWhileLoop.java +++ b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/control/DoWhileLoop.java @@ -110,7 +110,7 @@ public void accept(MethodVisitor visitor, MethodGenerationContext generationCont @Override public List getChildNodes() { - return ImmutableList.of(body, condition); + return ImmutableList.of(body, condition); } @Override diff --git a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/AndBytecodeExpression.java b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/AndBytecodeExpression.java index 8ce16dfc6e0a..946ac856d078 100644 --- a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/AndBytecodeExpression.java +++ b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/AndBytecodeExpression.java @@ -60,7 +60,7 @@ public BytecodeNode getBytecode(MethodGenerationContext generationContext) @Override public List getChildNodes() { - return ImmutableList.of(left, right); + return ImmutableList.of(left, right); } @Override diff --git a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/ArithmeticBytecodeExpression.java b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/ArithmeticBytecodeExpression.java index 82206ed3c38e..a5e24ecdbe06 100644 --- a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/ArithmeticBytecodeExpression.java +++ b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/ArithmeticBytecodeExpression.java @@ -193,7 +193,7 @@ public BytecodeNode getBytecode(MethodGenerationContext generationContext) @Override public List getChildNodes() { - return ImmutableList.of(left, right); + return ImmutableList.of(left, right); } @Override diff --git a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/ArrayLengthBytecodeExpression.java b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/ArrayLengthBytecodeExpression.java index 52591dba1d46..1bd8dec7a5a0 100644 --- a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/ArrayLengthBytecodeExpression.java +++ b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/ArrayLengthBytecodeExpression.java @@ -52,6 +52,6 @@ protected String formatOneLine() @Override public List getChildNodes() { - return ImmutableList.of(); + return ImmutableList.of(); } } diff --git a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/CastBytecodeExpression.java b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/CastBytecodeExpression.java index a8a523b9b81d..6a8360665486 100644 --- a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/CastBytecodeExpression.java +++ b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/CastBytecodeExpression.java @@ -320,7 +320,7 @@ protected String formatOneLine() @Override public List getChildNodes() { - return ImmutableList.of(instance); + return ImmutableList.of(instance); } private enum TypeKind { diff --git a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/ComparisonBytecodeExpression.java b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/ComparisonBytecodeExpression.java index 05277fa083e1..b964ac51cecc 100644 --- a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/ComparisonBytecodeExpression.java +++ b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/ComparisonBytecodeExpression.java @@ -304,7 +304,7 @@ public BytecodeNode getBytecode(MethodGenerationContext generationContext) @Override public List getChildNodes() { - return ImmutableList.of(left, right); + return ImmutableList.of(left, right); } @Override diff --git a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/GetElementBytecodeExpression.java b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/GetElementBytecodeExpression.java index 11cc6198790a..2dc7aea81645 100644 --- a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/GetElementBytecodeExpression.java +++ b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/GetElementBytecodeExpression.java @@ -60,6 +60,6 @@ protected String formatOneLine() @Override public List getChildNodes() { - return ImmutableList.of(index); + return ImmutableList.of(index); } } diff --git a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/GetFieldBytecodeExpression.java b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/GetFieldBytecodeExpression.java index d1eec06e19b1..4d5fd851642b 100644 --- a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/GetFieldBytecodeExpression.java +++ b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/GetFieldBytecodeExpression.java @@ -102,7 +102,7 @@ protected String formatOneLine() @Override public List getChildNodes() { - return (instance == null) ? ImmutableList.of() : ImmutableList.of(instance); + return (instance == null) ? ImmutableList.of() : ImmutableList.of(instance); } private static Field getDeclaredField(Class declaringClass, String name) diff --git a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/InlineIfBytecodeExpression.java b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/InlineIfBytecodeExpression.java index 630d52f68ef7..16d856d51da0 100644 --- a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/InlineIfBytecodeExpression.java +++ b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/InlineIfBytecodeExpression.java @@ -60,7 +60,7 @@ public BytecodeNode getBytecode(MethodGenerationContext generationContext) @Override public List getChildNodes() { - return ImmutableList.of(condition, ifTrue, ifFalse); + return ImmutableList.of(condition, ifTrue, ifFalse); } @Override diff --git a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/NegateBytecodeExpression.java b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/NegateBytecodeExpression.java index 392a7cc1c063..b93fc4c2aa50 100644 --- a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/NegateBytecodeExpression.java +++ b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/NegateBytecodeExpression.java @@ -57,7 +57,7 @@ public BytecodeNode getBytecode(MethodGenerationContext generationContext) @Override public List getChildNodes() { - return ImmutableList.of(value); + return ImmutableList.of(value); } @Override diff --git a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/NewInstanceBytecodeExpression.java b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/NewInstanceBytecodeExpression.java index 0a1ba4989257..762c1a4e8cbe 100644 --- a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/NewInstanceBytecodeExpression.java +++ b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/NewInstanceBytecodeExpression.java @@ -62,6 +62,6 @@ protected String formatOneLine() @Override public List getChildNodes() { - return ImmutableList.copyOf(parameters); + return ImmutableList.copyOf(parameters); } } diff --git a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/NotBytecodeExpression.java b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/NotBytecodeExpression.java index cc8ef09a7e92..59368f2dd3ee 100644 --- a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/NotBytecodeExpression.java +++ b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/NotBytecodeExpression.java @@ -54,7 +54,7 @@ public BytecodeNode getBytecode(MethodGenerationContext generationContext) @Override public List getChildNodes() { - return ImmutableList.of(value); + return ImmutableList.of(value); } @Override diff --git a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/OrBytecodeExpression.java b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/OrBytecodeExpression.java index 827a17b673bc..3634089027ac 100644 --- a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/OrBytecodeExpression.java +++ b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/OrBytecodeExpression.java @@ -60,7 +60,7 @@ public BytecodeNode getBytecode(MethodGenerationContext generationContext) @Override public List getChildNodes() { - return ImmutableList.of(left, right); + return ImmutableList.of(left, right); } @Override diff --git a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/PopBytecodeExpression.java b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/PopBytecodeExpression.java index d52903c08d2c..5c0c3922a4a4 100644 --- a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/PopBytecodeExpression.java +++ b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/PopBytecodeExpression.java @@ -51,6 +51,6 @@ protected String formatOneLine() @Override public List getChildNodes() { - return ImmutableList.of(instance); + return ImmutableList.of(instance); } } diff --git a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/ReturnBytecodeExpression.java b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/ReturnBytecodeExpression.java index 5b82cda194d4..ffbeed0f80c2 100644 --- a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/ReturnBytecodeExpression.java +++ b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/ReturnBytecodeExpression.java @@ -55,7 +55,7 @@ protected String formatOneLine() @Override public List getChildNodes() { - return ImmutableList.of(instance); + return ImmutableList.of(instance); } private static OpCode returnOpCode(ParameterizedType componentType) diff --git a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/SetArrayElementBytecodeExpression.java b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/SetArrayElementBytecodeExpression.java index 3ed46bb69057..cf07feddd334 100644 --- a/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/SetArrayElementBytecodeExpression.java +++ b/presto-bytecode/src/main/java/com/facebook/presto/bytecode/expression/SetArrayElementBytecodeExpression.java @@ -69,6 +69,6 @@ protected String formatOneLine() @Override public List getChildNodes() { - return ImmutableList.of(index, value); + return ImmutableList.of(index, value); } } diff --git a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraClientModule.java b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraClientModule.java index 7c02893b4900..10af5b19e579 100644 --- a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraClientModule.java +++ b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/CassandraClientModule.java @@ -103,6 +103,7 @@ public static CassandraSession createCassandraSession( List contactPoints = requireNonNull(config.getContactPoints(), "contactPoints is null"); checkArgument(!contactPoints.isEmpty(), "empty contactPoints"); + clusterBuilder.addContactPoints(contactPoints.toArray(new String[contactPoints.size()])); clusterBuilder.withPort(config.getNativeProtocolPort()); clusterBuilder.withReconnectionPolicy(new ExponentialReconnectionPolicy(500, 10000)); @@ -157,7 +158,6 @@ public static CassandraSession createCassandraSession( return new CassandraSession( connectorId.toString(), - contactPoints, clusterBuilder, config.getFetchSizeForPartitionKeySelect(), config.getLimitForPartitionKeySelect(), 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 6a629380e6b9..ca3a864528c0 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 @@ -76,7 +76,6 @@ public class CassandraSession private LoadingCache sessionBySchema; public CassandraSession(String connectorId, - final List contactPoints, final Builder clusterBuilder, int fetchSizeForPartitionKeySelect, int limitForPartitionKeySelect, @@ -96,7 +95,6 @@ public CassandraSession(String connectorId, public Session load(String key) throws Exception { - clusterBuilder.addContactPoints(contactPoints.toArray(new String[contactPoints.size()])); return clusterBuilder.build().connect(); } }); diff --git a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/RetryDriver.java b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/RetryDriver.java index 10eb4a9d9b1e..8a31891b4b58 100644 --- a/presto-cassandra/src/main/java/com/facebook/presto/cassandra/RetryDriver.java +++ b/presto-cassandra/src/main/java/com/facebook/presto/cassandra/RetryDriver.java @@ -46,7 +46,7 @@ private RetryDriver(int maxRetryAttempts, Duration sleepTime, Duration maxRetryT private RetryDriver() { - this(DEFAULT_RETRY_ATTEMPTS, DEFAULT_SLEEP_TIME, DEFAULT_MAX_RETRY_TIME, ImmutableList.of()); + this(DEFAULT_RETRY_ATTEMPTS, DEFAULT_SLEEP_TIME, DEFAULT_MAX_RETRY_TIME, ImmutableList.>of()); } public static RetryDriver retry() diff --git a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/MockCassandraSession.java b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/MockCassandraSession.java index c1f0ba8481ed..1e302cadd0ec 100644 --- a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/MockCassandraSession.java +++ b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/MockCassandraSession.java @@ -46,7 +46,6 @@ public class MockCassandraSession public MockCassandraSession(String connectorId, CassandraClientConfig config) { super(connectorId, - ImmutableList.of(), null, config.getFetchSizeForPartitionKeySelect(), config.getLimitForPartitionKeySelect(), 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 57b0cfb5bc0f..a727cb1129d0 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 @@ -100,16 +100,16 @@ public void testShowColumns() { MaterializedResult actual = computeActual("SHOW COLUMNS FROM orders"); - MaterializedResult expectedParametrizedVarchar = resultBuilder(getSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) - .row("orderkey", "bigint", "", "") - .row("custkey", "bigint", "", "") - .row("orderstatus", "varchar", "", "") - .row("totalprice", "double", "", "") - .row("orderdate", "varchar", "", "") - .row("orderpriority", "varchar", "", "") - .row("clerk", "varchar", "", "") - .row("shippriority", "integer", "", "") - .row("comment", "varchar", "", "") + MaterializedResult expectedParametrizedVarchar = resultBuilder(getSession(), VARCHAR, VARCHAR, VARCHAR) + .row("orderkey", "bigint", "") + .row("custkey", "bigint", "") + .row("orderstatus", "varchar", "") + .row("totalprice", "double", "") + .row("orderdate", "varchar", "") + .row("orderpriority", "varchar", "") + .row("clerk", "varchar", "") + .row("shippriority", "integer", "") + .row("comment", "varchar", "") .build(); assertEquals(actual, expectedParametrizedVarchar); diff --git a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestJsonCassandraHandles.java b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestJsonCassandraHandles.java index 7b114437322c..134c4dca798b 100644 --- a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestJsonCassandraHandles.java +++ b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/TestJsonCassandraHandles.java @@ -30,7 +30,7 @@ @Test public class TestJsonCassandraHandles { - private static final Map TABLE_HANDLE_AS_MAP = ImmutableMap.of( + private static final Map TABLE_HANDLE_AS_MAP = ImmutableMap.of( "connectorId", "cassandra", "schemaName", "cassandra_schema", "tableName", "cassandra_table"); diff --git a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/util/TestHostAddressFactory.java b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/util/TestHostAddressFactory.java index e4799843385c..fb341506f83d 100644 --- a/presto-cassandra/src/test/java/com/facebook/presto/cassandra/util/TestHostAddressFactory.java +++ b/presto-cassandra/src/test/java/com/facebook/presto/cassandra/util/TestHostAddressFactory.java @@ -32,7 +32,7 @@ public class TestHostAddressFactory public void testToHostAddressList() throws Exception { - Set hosts = ImmutableSet.of( + Set hosts = ImmutableSet.of( new TestHost( new InetSocketAddress( InetAddress.getByAddress(new byte[] { diff --git a/presto-cli/src/main/java/com/facebook/presto/cli/AlignedTablePrinter.java b/presto-cli/src/main/java/com/facebook/presto/cli/AlignedTablePrinter.java index 8f8ff81c4a2f..9811a9f3fc46 100644 --- a/presto-cli/src/main/java/com/facebook/presto/cli/AlignedTablePrinter.java +++ b/presto-cli/src/main/java/com/facebook/presto/cli/AlignedTablePrinter.java @@ -57,7 +57,7 @@ public AlignedTablePrinter(List fieldNames, Writer writer) public void finish() throws IOException { - printRows(ImmutableList.of(), true); + printRows(ImmutableList.>of(), true); writer.append(format("(%s row%s)%n", rowCount, (rowCount != 1) ? "s" : "")); writer.flush(); } 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 1382d61cb6d5..5bcb3351d109 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 @@ -55,7 +55,7 @@ public class ClientOptions @Option(name = "--krb5-config-path", title = "krb5 config path", description = "Kerberos config file path (default: /etc/krb5.conf)") public String krb5ConfigPath = "/etc/krb5.conf"; - @Option(name = "--krb5-keytab-path", title = "krb5 keytab path", description = "Kerberos key table path (default: /etc/krb5.keytab)") + @Option(name = "--krb5-keytab-path", title = "krb5 keytab path", description = "Kerberos key table path") public String krb5KeytabPath = "/etc/krb5.keytab"; @Option(name = "--krb5-credential-cache-path", title = "krb5 credential cache path", description = "Kerberos credential cache path") @@ -98,7 +98,7 @@ public class ClientOptions @Option(name = "--debug", title = "debug", description = "Enable debug information") public boolean debug; - @Option(name = "--log-levels-file", title = "log levels file", description = "Configure log levels for debugging using this file") + @Option(name = "--log-levels-file", title = "log levels", description = "Configure log levels for debugging") public String logLevelsFile; @Option(name = "--execute", title = "execute", description = "Execute specified statements and exit") diff --git a/presto-cli/src/main/java/com/facebook/presto/cli/CsvPrinter.java b/presto-cli/src/main/java/com/facebook/presto/cli/CsvPrinter.java index b42f2c56acce..cae18a6db2be 100644 --- a/presto-cli/src/main/java/com/facebook/presto/cli/CsvPrinter.java +++ b/presto-cli/src/main/java/com/facebook/presto/cli/CsvPrinter.java @@ -59,7 +59,7 @@ public void printRows(List> rows, boolean complete) public void finish() throws IOException { - printRows(ImmutableList.of(), true); + printRows(ImmutableList.>of(), true); writer.flush(); checkError(); } diff --git a/presto-cli/src/main/java/com/facebook/presto/cli/PerfTest.java b/presto-cli/src/main/java/com/facebook/presto/cli/PerfTest.java index 3fdae634be27..70c0c84baa30 100644 --- a/presto-cli/src/main/java/com/facebook/presto/cli/PerfTest.java +++ b/presto-cli/src/main/java/com/facebook/presto/cli/PerfTest.java @@ -163,7 +163,7 @@ public ParallelQueryRunner(int maxParallelism, URI server, String catalog, Strin schema, TimeZone.getDefault().getID(), Locale.getDefault(), - ImmutableMap.of(), + ImmutableMap.of(), null, debug, clientRequestTimeout); 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 63ba5aafaa6b..1aad539b45bb 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 @@ -20,7 +20,9 @@ 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.jetty.JettyIoPool; import io.airlift.http.client.spnego.KerberosConfig; import io.airlift.json.JsonCodec; import io.airlift.units.Duration; @@ -66,8 +68,8 @@ public QueryRunner( kerberosRemoteServiceName, authenticationEnabled), kerberosConfig, - Optional.empty(), - ImmutableList.of()); + Optional.empty(), + ImmutableList.of()); } public ClientSession getSession() diff --git a/presto-cli/src/main/java/com/facebook/presto/cli/TsvPrinter.java b/presto-cli/src/main/java/com/facebook/presto/cli/TsvPrinter.java index 1f9f01e8101a..d0414e7f1034 100644 --- a/presto-cli/src/main/java/com/facebook/presto/cli/TsvPrinter.java +++ b/presto-cli/src/main/java/com/facebook/presto/cli/TsvPrinter.java @@ -44,7 +44,7 @@ public void printRows(List> rows, boolean complete) { if (needHeader) { needHeader = false; - printRows(ImmutableList.of(fieldNames), false); + printRows(ImmutableList.>of(fieldNames), false); } for (List row : rows) { @@ -56,7 +56,7 @@ public void printRows(List> rows, boolean complete) public void finish() throws IOException { - printRows(ImmutableList.of(), true); + printRows(ImmutableList.>of(), true); writer.flush(); } 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 3d6fdac8af2d..36ab9ea51867 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 @@ -15,6 +15,7 @@ import com.facebook.presto.client.ClientSession; import com.google.common.collect.ImmutableList; +import com.google.common.net.HostAndPort; import org.testng.annotations.Test; import java.util.Optional; @@ -28,13 +29,13 @@ public void testAutoCompleteWithoutSchema() { ClientSession session = new ClientOptions().toClientSession(); QueryRunner runner = QueryRunner.create(session, - Optional.empty(), - Optional.empty(), - Optional.empty(), - Optional.empty(), - Optional.empty(), - Optional.empty(), - Optional.empty(), + Optional.empty(), + Optional.empty(), + Optional.empty(), + Optional.empty(), + Optional.empty(), + Optional.empty(), + Optional.empty(), false, null); TableNameCompleter completer = new TableNameCompleter(runner); diff --git a/presto-docs/Makefile b/presto-docs/Makefile index cf89add56332..30612cd5fa73 100644 --- a/presto-docs/Makefile +++ b/presto-docs/Makefile @@ -16,7 +16,7 @@ endif # Internal variables. PAPEROPT_a4 = -D latex_paper_size=a4 PAPEROPT_letter = -D latex_paper_size=letter -ALLSPHINXOPTS = -n -W -d $(BUILDDIR)/doctrees $(PAPEROPT_$(PAPER)) $(SPHINXOPTS) $(SOURCEDIR) +ALLSPHINXOPTS = -W -d $(BUILDDIR)/doctrees $(PAPEROPT_$(PAPER)) $(SPHINXOPTS) $(SOURCEDIR) # the i18n builder cannot share the environment and doctrees with the others I18NSPHINXOPTS = $(PAPEROPT_$(PAPER)) $(SPHINXOPTS) $(SOURCEDIR) diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 39d843f4c136..33c7c59d1464 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -45,7 +45,7 @@ - io.airlift.maven.plugins + kr.motd.maven sphinx-maven-plugin true diff --git a/presto-docs/src/main/sphinx/connector/accumulo.rst b/presto-docs/src/main/sphinx/connector/accumulo.rst index 6c77836abf8c..7e4b274dfa1e 100644 --- a/presto-docs/src/main/sphinx/connector/accumulo.rst +++ b/presto-docs/src/main/sphinx/connector/accumulo.rst @@ -100,12 +100,12 @@ Simply issue a ``CREATE TABLE`` statement to create a new Presto/Accumulo table: .. code-block:: none - Column | Type | Extra | Comment - -----------+---------+-------+--------------------------------------------------- - recordkey | varchar | | Accumulo row ID - name | varchar | | Accumulo column name:name. Indexed: false - age | bigint | | Accumulo column age:age. Indexed: false - birthday | date | | Accumulo column birthday:birthday. Indexed: false + Column | Type | Comment + -----------+---------+--------------------------------------------------- + recordkey | varchar | Accumulo row ID + name | varchar | Accumulo column name:name. Indexed: false + age | bigint | Accumulo column age:age. Indexed: false + birthday | date | Accumulo column birthday:birthday. Indexed: false This command will create a new Accumulo table with the ``recordkey`` column as the Accumulo row ID. The name, age, and birthday columns are mapped to @@ -146,12 +146,12 @@ For example: .. code-block:: none - Column | Type | Extra | Comment - -----------+---------+-------+----------------------------------------------- - recordkey | varchar | | Accumulo row ID - name | varchar | | Accumulo column metadata:name. Indexed: false - age | bigint | | Accumulo column metadata:age. Indexed: false - birthday | date | | Accumulo column metadata:date. Indexed: false + Column | Type | Comment + -----------+---------+----------------------------------------------- + recordkey | varchar | Accumulo row ID + name | varchar | Accumulo column metadata:name. Indexed: false + age | bigint | Accumulo column metadata:age. Indexed: false + birthday | date | Accumulo column metadata:date. Indexed: false You can then issue ``INSERT`` statements to put data into Accumulo. @@ -665,11 +665,11 @@ when creating the external table. .. code-block:: none - Column | Type | Extra | Comment - --------+---------+-------+------------------------------------- - a | varchar | | Accumulo row ID - b | bigint | | Accumulo column b:b. Indexed: true - c | date | | Accumulo column c:c. Indexed: true + Column | Type | Comment + --------+---------+------------------------------------- + a | varchar | Accumulo row ID + b | bigint | Accumulo column b:b. Indexed: true + c | date | Accumulo column c:c. Indexed: true 2. Using the ZooKeeper CLI, delete the corresponding znode. Note this uses the default ZooKeeper metadata root of ``/presto-accumulo`` diff --git a/presto-docs/src/main/sphinx/connector/cassandra.rst b/presto-docs/src/main/sphinx/connector/cassandra.rst index 46f4cebf1e3d..081cd62d0ba1 100644 --- a/presto-docs/src/main/sphinx/connector/cassandra.rst +++ b/presto-docs/src/main/sphinx/connector/cassandra.rst @@ -182,11 +182,11 @@ This table can be described in Presto:: .. code-block:: none - Column | Type | Extra | Comment - ---------+---------+-------+--------- - user_id | bigint | | - fname | varchar | | - lname | varchar | | + Column | Type | Null | Partition Key | Comment + ---------+---------+------+---------------+--------- + user_id | bigint | true | true | + fname | varchar | true | false | + lname | varchar | true | false | (3 rows) This table can then be queried in Presto:: diff --git a/presto-docs/src/main/sphinx/connector/hive-security.rst b/presto-docs/src/main/sphinx/connector/hive-security.rst index cb09f5a7add4..10c482e4f21b 100644 --- a/presto-docs/src/main/sphinx/connector/hive-security.rst +++ b/presto-docs/src/main/sphinx/connector/hive-security.rst @@ -191,7 +191,7 @@ Example configuration with ``KERBEROS`` authentication .. code-block:: none hive.metastore.authentication.type=KERBEROS - hive.metastore.service.principal=hive/hive-metastore-host.example.com@EXAMPLE.COM + hive.metastore.principal=hive/hive-metastore-host.example.com@EXAMPLE.COM hive.metastore.client.principal=presto@EXAMPLE.COM hive.metastore.client.keytab=/etc/presto/hive.keytab @@ -346,7 +346,7 @@ section :ref:`configuring-hadoop-impersonation`. Kerberos is not used. .. code-block:: none hive.hdfs.authentication.type=KERBEROS - hive.hdfs.impersonation.enabled=true + hive.hdfs.impersonation=true hive.hdfs.presto.principal=presto@EXAMPLE.COM hive.hdfs.presto.keytab=/etc/presto/hdfs.keytab diff --git a/presto-docs/src/main/sphinx/connector/hive.rst b/presto-docs/src/main/sphinx/connector/hive.rst index 3ed8d543c576..b9fdee925b64 100644 --- a/presto-docs/src/main/sphinx/connector/hive.rst +++ b/presto-docs/src/main/sphinx/connector/hive.rst @@ -400,7 +400,7 @@ existing data in S3:: Drop the external table ``request_logs``. This only drops the metadata for the table. The referenced data directory is not deleted:: - DROP TABLE hive.web.request_logs + DROP hive.web.request_logs Drop a schema:: diff --git a/presto-docs/src/main/sphinx/connector/jmx.rst b/presto-docs/src/main/sphinx/connector/jmx.rst index 266c9e5cf3c0..25aeed9895fb 100644 --- a/presto-docs/src/main/sphinx/connector/jmx.rst +++ b/presto-docs/src/main/sphinx/connector/jmx.rst @@ -21,11 +21,10 @@ To configure the JMX connector, create a catalog properties file connector.name=jmx -To enable periodical dumps, define the following properties: +To enable periodical dumps, define following properties: .. code-block:: none - connector.name=jmx jmx.dump-tables=java.lang:type=Runtime,com.facebook.presto.execution.scheduler:name=NodeScheduler jmx.dump-period=10s jmx.max-entries=86400 @@ -35,15 +34,6 @@ which MBeans will be sampled and stored in memory every ``dump-period``. History will have limited size of ``max-entries`` of entries. Both ``dump-period`` and ``max-entries`` have default values of ``10s`` and ``86400`` accordingly. -Commas in MBean names should be escaped in the following manner: - -.. code-block:: none - - connector.name=jmx - jmx.dump-tables=com.facebook.presto.memory:type=memorypool\\,name=general,\ - com.facebook.presto.memory:type=memorypool\\,name=system,\ - com.facebook.presto.memory:type=memorypool\\,name=reserved - Querying JMX ------------ diff --git a/presto-docs/src/main/sphinx/connector/kafka-tutorial.rst b/presto-docs/src/main/sphinx/connector/kafka-tutorial.rst index d4d0e820e1dd..3c74e47f4980 100644 --- a/presto-docs/src/main/sphinx/connector/kafka-tutorial.rst +++ b/presto-docs/src/main/sphinx/connector/kafka-tutorial.rst @@ -150,19 +150,19 @@ built-in ones: .. code-block:: none presto:tpch> DESCRIBE customer; - Column | Type | Extra | Comment - -------------------+---------+-------+--------------------------------------------- - _partition_id | bigint | | Partition Id - _partition_offset | bigint | | Offset for the message within the partition - _segment_start | bigint | | Segment start offset - _segment_end | bigint | | Segment end offset - _segment_count | bigint | | Running message count per segment - _key | varchar | | Key text - _key_corrupt | boolean | | Key data is corrupt - _key_length | bigint | | Total number of key bytes - _message | varchar | | Message text - _message_corrupt | boolean | | Message data is corrupt - _message_length | bigint | | Total number of message bytes + Column | Type | Null | Partition Key | Comment + -------------------+---------+------+---------------+--------------------------------------------- + _partition_id | bigint | true | false | Partition Id + _partition_offset | bigint | true | false | Offset for the message within the partition + _segment_start | bigint | true | false | Segment start offset + _segment_end | bigint | true | false | Segment end offset + _segment_count | bigint | true | false | Running message count per segment + _key | varchar | true | false | Key text + _key_corrupt | boolean | true | false | Key data is corrupt + _key_length | bigint | true | false | Total number of key bytes + _message | varchar | true | false | Message text + _message_corrupt | boolean | true | false | Message data is corrupt + _message_length | bigint | true | false | Total number of message bytes (11 rows) presto:tpch> SELECT count(*) FROM customer; @@ -226,20 +226,20 @@ The customer table now has an additional column: ``kafka_key``. .. code-block:: none presto:tpch> DESCRIBE customer; - Column | Type | Extra | Comment - -------------------+---------+-------+--------------------------------------------- - kafka_key | bigint | | - _partition_id | bigint | | Partition Id - _partition_offset | bigint | | Offset for the message within the partition - _segment_start | bigint | | Segment start offset - _segment_end | bigint | | Segment end offset - _segment_count | bigint | | Running message count per segment - _key | varchar | | Key text - _key_corrupt | boolean | | Key data is corrupt - _key_length | bigint | | Total number of key bytes - _message | varchar | | Message text - _message_corrupt | boolean | | Message data is corrupt - _message_length | bigint | | Total number of message bytes + Column | Type | Null | Partition Key | Comment + -------------------+---------+------+---------------+--------------------------------------------- + kafka_key | bigint | true | false | + _partition_id | bigint | true | false | Partition Id + _partition_offset | bigint | true | false | Offset for the message within the partition + _segment_start | bigint | true | false | Segment start offset + _segment_end | bigint | true | false | Segment end offset + _segment_count | bigint | true | false | Running message count per segment + _key | varchar | true | false | Key text + _key_corrupt | boolean | true | false | Key data is corrupt + _key_length | bigint | true | false | Total number of key bytes + _message | varchar | true | false | Message text + _message_corrupt | boolean | true | false | Message data is corrupt + _message_length | bigint | true | false | Total number of message bytes (12 rows) presto:tpch> SELECT kafka_key FROM customer ORDER BY kafka_key LIMIT 10; @@ -343,29 +343,29 @@ the sum query from earlier can operate on the ``account_balance`` column directl .. code-block:: none presto:tpch> DESCRIBE customer; - Column | Type | Extra | Comment - -------------------+---------+-------+--------------------------------------------- - kafka_key | bigint | | - row_number | bigint | | - customer_key | bigint | | - name | varchar | | - address | varchar | | - nation_key | bigint | | - phone | varchar | | - account_balance | double | | - market_segment | varchar | | - comment | varchar | | - _partition_id | bigint | | Partition Id - _partition_offset | bigint | | Offset for the message within the partition - _segment_start | bigint | | Segment start offset - _segment_end | bigint | | Segment end offset - _segment_count | bigint | | Running message count per segment - _key | varchar | | Key text - _key_corrupt | boolean | | Key data is corrupt - _key_length | bigint | | Total number of key bytes - _message | varchar | | Message text - _message_corrupt | boolean | | Message data is corrupt - _message_length | bigint | | Total number of message bytes + Column | Type | Null | Partition Key | Comment + -------------------+---------+------+---------------+--------------------------------------------- + kafka_key | bigint | true | false | + row_number | bigint | true | false | + customer_key | bigint | true | false | + name | varchar | true | false | + address | varchar | true | false | + nation_key | bigint | true | false | + phone | varchar | true | false | + account_balance | double | true | false | + market_segment | varchar | true | false | + comment | varchar | true | false | + _partition_id | bigint | true | false | Partition Id + _partition_offset | bigint | true | false | Offset for the message within the partition + _segment_start | bigint | true | false | Segment start offset + _segment_end | bigint | true | false | Segment end offset + _segment_count | bigint | true | false | Running message count per segment + _key | varchar | true | false | Key text + _key_corrupt | boolean | true | false | Key data is corrupt + _key_length | bigint | true | false | Total number of key bytes + _message | varchar | true | false | Message text + _message_corrupt | boolean | true | false | Message data is corrupt + _message_length | bigint | true | false | Total number of message bytes (21 rows) presto:tpch> SELECT * FROM customer LIMIT 5; diff --git a/presto-docs/src/main/sphinx/functions.rst b/presto-docs/src/main/sphinx/functions.rst index e2842121a6d6..4d42a10e0539 100644 --- a/presto-docs/src/main/sphinx/functions.rst +++ b/presto-docs/src/main/sphinx/functions.rst @@ -23,5 +23,4 @@ Functions and Operators functions/color functions/array functions/map - functions/lambda functions/teradata diff --git a/presto-docs/src/main/sphinx/functions/aggregate.rst b/presto-docs/src/main/sphinx/functions/aggregate.rst index 10bb2392493e..ba507b33a255 100644 --- a/presto-docs/src/main/sphinx/functions/aggregate.rst +++ b/presto-docs/src/main/sphinx/functions/aggregate.rst @@ -65,8 +65,8 @@ General Aggregate Functions .. function:: max_by(x, y, n) -> array<[same as x]> - Returns ``n`` values of ``x`` associated with the ``n`` largest of all input values of ``y`` - in descending order of ``y``. + Returns ``n`` values of ``x`` associated with the ``n`` largest of all input values of ``y``, + in non-ascending order of ``y``. .. function:: min_by(x, y) -> [same as x] @@ -74,8 +74,8 @@ General Aggregate Functions .. function:: min_by(x, y, n) -> array<[same as x]> - Returns ``n`` values of ``x`` associated with the ``n`` smallest of all input values of ``y`` - in ascending order of ``y``. + Returns ``n`` values of ``x`` associated with the ``n`` smallest of all input values of ``y``, + in non-descending order of ``y``. .. function:: max(x) -> [same as input] diff --git a/presto-docs/src/main/sphinx/functions/array.rst b/presto-docs/src/main/sphinx/functions/array.rst index 7c5a2e3f7664..239db77bf539 100644 --- a/presto-docs/src/main/sphinx/functions/array.rst +++ b/presto-docs/src/main/sphinx/functions/array.rst @@ -78,20 +78,10 @@ Array Functions If ``index`` >= 0, this function provides the same functionality as the SQL-standard subscript operator (``[]``). If ``index`` < 0, ``element_at`` accesses elements from the last to the first. -.. function:: filter(array, function) -> array - :noindex: - - See :func:`filter`. - .. function:: flatten(x) -> array Flattens an ``array(array(T))`` to an ``array(T)`` by concatenating the contained arrays. -.. function:: reduce(array, initialState, inputFunction, outputFunction) -> x - :noindex: - - See :func:`reduce`. - .. function:: reverse(x) -> array :noindex: @@ -120,11 +110,6 @@ Array Functions Subsets array ``x`` starting from index ``start`` (or starting from the end if ``start`` is negative) with a length of ``length``. -.. function:: transform(array, function) -> array - :noindex: - - See :func:`transform`. - .. function:: zip(array1, array2[, ...]) -> array Merges the given arrays, element-wise, into a single array of rows. The M-th element of diff --git a/presto-docs/src/main/sphinx/functions/lambda.rst b/presto-docs/src/main/sphinx/functions/lambda.rst deleted file mode 100644 index ac65e5e4e4f9..000000000000 --- a/presto-docs/src/main/sphinx/functions/lambda.rst +++ /dev/null @@ -1,74 +0,0 @@ -================================ -Lambda Expressions and Functions -================================ - -Lambda Expression ------------------ - -Lambda expressions are written with ``->``:: - - x -> x + 1 - (x, y) -> x + y - x -> regexp_like(x, 'a+') - x -> x[1] / x[2] - x -> IF(x > 0, x, -x) - x -> COALESCE(x, 0) - x -> CAST(x AS JSON) - -Most SQL expressions can be used in a lambda body, with a few exceptions: - -* Subqueries are not supported. -* The ``TRY`` function is not supported yet. (:func:`try_cast` is supported.) -* Capture is not supported yet: - - * Columns or relations cannot be referenced. - * Only lambda variables from the inner-most lambda expression can be referenced. - -Lambda Functions ----------------- - -.. function:: filter(array, function) -> ARRAY - - Constructs an array from those elements of ``array`` for which ``function`` returns true:: - - SELECT filter(ARRAY [], x -> true); -- [] - SELECT filter(ARRAY [5, -6, NULL, 7], x -> x > 0); -- [5, 7] - SELECT filter(ARRAY [5, NULL, 7, NULL], x -> x IS NOT NULL); -- [5, 7] - -.. function:: map_filter(map, function) -> MAP - - Constructs a map from those entries of ``map`` for which ``function`` returns true:: - - SELECT map_filter(MAP(ARRAY[], ARRAY[]), (k, v) -> true); -- {} - SELECT map_filter(MAP(ARRAY[10, 20, 30], ARRAY['a', NULL, 'c']), (k, v) -> v IS NOT NULL); -- {10 -> a, 30 -> c} - SELECT map_filter(MAP(ARRAY['k1', 'k2', 'k3'], ARRAY[20, 3, 15]), (k, v) -> v > 10); -- {k1 -> 20, k3 -> 15} - -.. function:: transform(array, function) -> ARRAY - - Returns an array that applies ``function`` to each element of ``array``:: - - SELECT transform(ARRAY [], x -> x + 1); -- [] - SELECT transform(ARRAY [5, 6], x -> x + 1); -- [6, 7] - SELECT transform(ARRAY [5, NULL, 6], x -> COALESCE(x, 0) + 1); -- [6, 1, 7] - SELECT transform(ARRAY ['x', 'abc', 'z'], x -> x || '0'); -- ['x0', 'abc0', 'z0'] - SELECT transform(ARRAY [ARRAY [1, NULL, 2], ARRAY[3, NULL]], a -> filter(a, x -> x IS NOT NULL)); -- [[1, 2], [3]] - -.. function:: reduce(array, initialState S, inputFunction, outputFunction) -> R - - Returns a single value reduced from ``array``. ``inputFunction`` will - be invoked for each element in ``array`` in order. In addition to taking - the element, ``inputFunction`` takes the current state, initially - ``initialState``, and returns the new state. ``outputFunction`` will be - invoked to turn the final state into the result value. It may be identity - function (``i -> i``). For example:: - - SELECT reduce(ARRAY [], 0, (s, x) -> s + x, s -> s); -- 0 - SELECT reduce(ARRAY [5, 20, 50], 0, (s, x) -> s + x, s -> s); -- 75 - SELECT reduce(ARRAY [5, 20, NULL, 50], 0, (s, x) -> s + x, s -> s); -- NULL - SELECT reduce(ARRAY [5, 20, NULL, 50], 0, (s, x) -> s + COALESCE(x, 0), s -> s); -- 75 - SELECT reduce(ARRAY [5, 20, NULL, 50], 0, (s, x) -> IF(x IS NULL, s, s + x), s -> s); -- 75 - SELECT reduce(ARRAY [2147483647, 1], CAST (0 AS BIGINT), (s, x) -> s + x, s -> s); -- 2147483648 - SELECT reduce(ARRAY [5, 6, 10, 20], -- calculates arithmetic average: 10.25 - CAST(ROW(0.0, 0) AS ROW(sum DOUBLE, count INTEGER)), - (s, x) -> CAST(ROW(x + s.sum, s.count + 1) AS ROW(sum DOUBLE, count INTEGER)), - s -> IF(s.count = 0, NULL, s.sum / s.count)); diff --git a/presto-docs/src/main/sphinx/functions/map.rst b/presto-docs/src/main/sphinx/functions/map.rst index 2f5ade7ef498..2f36329a75fa 100644 --- a/presto-docs/src/main/sphinx/functions/map.rst +++ b/presto-docs/src/main/sphinx/functions/map.rst @@ -22,12 +22,6 @@ Map Functions Returns value for given ``key``, or ``NULL`` if the key is not contained in the map. -.. function:: map() -> map - - Returns an empty map. :: - - SELECT map(); -- {} - .. function:: map(array, array) -> map Returns a map created using the given key/value arrays. :: @@ -41,11 +35,6 @@ Map Functions Returns the union of two maps. If a key is found in both ``x`` and ``y``, that key's value in the resulting map comes from ``y``. -.. function:: map_filter(map, function) -> map - :noindex: - - See :func:`map_filter`. - .. function:: map_keys(x) -> array Returns all the keys in the map ``x``. @@ -53,3 +42,4 @@ Map Functions .. function:: map_values(x) -> array Returns all the values in the map ``x``. + diff --git a/presto-docs/src/main/sphinx/language/types.rst b/presto-docs/src/main/sphinx/language/types.rst index d720b9b572d9..bc4676e14c4c 100644 --- a/presto-docs/src/main/sphinx/language/types.rst +++ b/presto-docs/src/main/sphinx/language/types.rst @@ -79,11 +79,7 @@ VARCHAR CHAR ---- - Fixed length character data. A CHAR type without length specified has a default length of 1. - A ``CHAR(x)`` value always has ``x`` characters. For instance, casting ``dog`` to ``CHAR(7)`` - adds 4 implicit trailing spaces. Leading and trailing spaces are included in comparisons of - CHAR values. As a result, two character values with different lengths (``CHAR(x)`` and - ``CHAR(y)`` where ``x != y``) will never be equal. + Fixed length character data. Char type without length specified has default length of 1. Example type definitions: ``char``, ``char(20)`` diff --git a/presto-docs/src/main/sphinx/release.rst b/presto-docs/src/main/sphinx/release.rst index b312e66f1ab2..3630a8dc6c84 100644 --- a/presto-docs/src/main/sphinx/release.rst +++ b/presto-docs/src/main/sphinx/release.rst @@ -5,11 +5,6 @@ Release Notes .. toctree:: :maxdepth: 1 - release/release-0.161 - release/release-0.160 - release/release-0.159 - release/release-0.158 - release/release-0.157.1 release/release-0.157 release/release-0.156 release/release-0.155 diff --git a/presto-docs/src/main/sphinx/release/release-0.157.1.rst b/presto-docs/src/main/sphinx/release/release-0.157.1.rst deleted file mode 100644 index cb6006d656d6..000000000000 --- a/presto-docs/src/main/sphinx/release/release-0.157.1.rst +++ /dev/null @@ -1,9 +0,0 @@ -=============== -Release 0.157.1 -=============== - -General Changes ---------------- - -* Fix regression that could cause high CPU and heap usage on coordinator, - when processing certain types of long running queries. diff --git a/presto-docs/src/main/sphinx/release/release-0.158.rst b/presto-docs/src/main/sphinx/release/release-0.158.rst deleted file mode 100644 index 484cbccbf4a3..000000000000 --- a/presto-docs/src/main/sphinx/release/release-0.158.rst +++ /dev/null @@ -1,36 +0,0 @@ -============= -Release 0.158 -============= - -General Changes ---------------- - -* Fix regression that could cause high CPU and heap usage on coordinator - when processing certain types of long running queries. -* Fix incorrect pruning of output columns in ``EXPLAIN ANALYZE``. -* Fix ordering of ``CHAR`` values so that trailing spaces are ordered after control characters. -* Fix query failures for connectors that produce non-remotely accessible splits. -* Fix non-linear performance issue when parsing certain SQL expressions. -* Fix case-sensitivity issues when operating on columns of ``ROW`` data type. -* Fix failure when creating views for tables names that need quoting. -* Return ``NULL`` from :func:`element_at` for out-of-range indices instead of failing. -* Remove redundancies in query plans, which can reduce data transfers over the network and reduce CPU requirements. -* Validate resource groups configuration file on startup to ensure that all - selectors reference a configured resource group. -* Add experimental on-disk merge sort for aggregations. This can be enabled with - the ``experimental.spill-enabled`` configuration flag. -* Push down predicates for ``DECIMAL``, ``TINYINT``, ``SMALLINT`` and ``REAL`` data types. - -Hive Changes ------------- - -* Add hidden ``$bucket`` column for bucketed tables that - contains the bucket number for the current row. -* Prevent inserting into non-managed (i.e., external) tables. -* Add configurable size limit to Hive metastore cache to avoid using too much - coordinator memory. - -Cassandra Changes ------------------ - -* Allow starting the server even if a contact point hostname cannot be resolved. diff --git a/presto-docs/src/main/sphinx/release/release-0.159.rst b/presto-docs/src/main/sphinx/release/release-0.159.rst deleted file mode 100644 index 2cd37878a3cd..000000000000 --- a/presto-docs/src/main/sphinx/release/release-0.159.rst +++ /dev/null @@ -1,15 +0,0 @@ -============= -Release 0.159 -============= - -General Changes ---------------- - -* Improve predicate performance for ``JOIN`` queries. - -Hive Changes ------------- - -* Optimize filtering of partition names to reduce object creation. -* Add limit on the number of partitions that can potentially be read per table scan. - This limit is configured using ``hive.max-partitions-per-scan`` and defaults to 100,000. diff --git a/presto-docs/src/main/sphinx/release/release-0.160.rst b/presto-docs/src/main/sphinx/release/release-0.160.rst deleted file mode 100644 index e459707a11f3..000000000000 --- a/presto-docs/src/main/sphinx/release/release-0.160.rst +++ /dev/null @@ -1,22 +0,0 @@ -============= -Release 0.160 -============= - -General Changes ---------------- - -* Fix planning failure when query has multiple unions with identical underlying columns. -* Fix planning failure when multiple ``IN`` predicates contain an identical subquery. -* Fix resource waste where coordinator floods rebooted workers if worker - comes back before coordinator times out the query. -* Add :doc:`/functions/lambda`. - -Hive Changes ------------- - -* Fix planning failure when inserting into columns of struct types with uppercase field names. -* Fix resource leak when using Kerberos authentication with impersonation. -* Fix creating external tables so that they are properly recognized by the Hive metastore. - The Hive table property ``EXTERNAL`` is now set to ``TRUE`` in addition to the setting - the table type. Any previously created tables need to be modified to have this property. -* Add ``bucket_execution_enabled`` session property. diff --git a/presto-docs/src/main/sphinx/release/release-0.161.rst b/presto-docs/src/main/sphinx/release/release-0.161.rst deleted file mode 100644 index 0c2bf17dae0d..000000000000 --- a/presto-docs/src/main/sphinx/release/release-0.161.rst +++ /dev/null @@ -1,37 +0,0 @@ -============= -Release 0.161 -============= - -General Changes ---------------- - -* Fix correctness issue for queries involving multiple nested EXCEPT clauses. - A query such as ``a EXCEPT (b EXCEPT c)`` was incorrectly evaluated as - ``a EXCEPT b EXCEPT c`` and thus could return the wrong result. -* Fix failure when executing prepared statements that contain parameters in the join criteria. -* Fix failure when describing the output of prepared statements that contain aggregations. -* Fix planning failure when a lambda is used in the context of an aggregation or subquery. -* Fix column resolution rules for ``ORDER BY`` to match the behavior expected - by the SQL standard. This is a change in semantics that breaks - backwards compatibility. To ease migration of existing queries, the legacy - behavior can be restored by the ``deprecated.legacy-order-by`` config option - or the ``legacy_order_by`` session property. -* Improve error message when coordinator responds with ``403 FORBIDDEN``. -* Improve performance for queries containing expressions in the join criteria - that reference columns on one side of the join. -* Improve performance of :func:`map_concat` when one argument is empty. -* Remove `/v1/execute` resource. -* Add new column to :doc:`/sql/show-columns` (and :doc:`/sql/describe`) - to show extra information from connectors. -* Add :func:`map` to construct an empty :ref:`map_type`. - -Hive Connector --------------- - -* Remove ``"Partition Key: "`` prefix from column comments and - replace it with the new extra information field described above. - -JMX Connector -------------- - -* Add support for escaped commas in ``jmx.dump-tables`` config property. diff --git a/presto-example-http/src/test/java/com/facebook/presto/example/ExampleHttpServer.java b/presto-example-http/src/test/java/com/facebook/presto/example/ExampleHttpServer.java index 1d78d2d600d3..b9b51c9ac302 100644 --- a/presto-example-http/src/test/java/com/facebook/presto/example/ExampleHttpServer.java +++ b/presto-example-http/src/test/java/com/facebook/presto/example/ExampleHttpServer.java @@ -75,7 +75,7 @@ private static class ExampleHttpServerModule @Override public void configure(Binder binder) { - binder.bind(new TypeLiteral>() {}).annotatedWith(TheServlet.class).toInstance(ImmutableMap.of()); + binder.bind(new TypeLiteral>() {}).annotatedWith(TheServlet.class).toInstance(ImmutableMap.of()); binder.bind(Servlet.class).annotatedWith(TheServlet.class).toInstance(new ExampleHttpServlet()); } } diff --git a/presto-example-http/src/test/java/com/facebook/presto/example/MetadataUtil.java b/presto-example-http/src/test/java/com/facebook/presto/example/MetadataUtil.java index 7d817ef67a88..cea4da8212bd 100644 --- a/presto-example-http/src/test/java/com/facebook/presto/example/MetadataUtil.java +++ b/presto-example-http/src/test/java/com/facebook/presto/example/MetadataUtil.java @@ -16,6 +16,7 @@ import com.facebook.presto.spi.type.StandardTypes; import com.facebook.presto.spi.type.Type; import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; import com.fasterxml.jackson.databind.deser.std.FromStringDeserializer; import com.google.common.collect.ImmutableMap; import io.airlift.json.JsonCodec; @@ -45,7 +46,7 @@ private MetadataUtil() static { ObjectMapperProvider objectMapperProvider = new ObjectMapperProvider(); - objectMapperProvider.setJsonDeserializers(ImmutableMap.of(Type.class, new TestingTypeDeserializer())); + objectMapperProvider.setJsonDeserializers(ImmutableMap., JsonDeserializer>of(Type.class, new TestingTypeDeserializer())); JsonCodecFactory codecFactory = new JsonCodecFactory(objectMapperProvider); CATALOG_CODEC = codecFactory.mapJsonCodec(String.class, listJsonCodec(ExampleTable.class)); TABLE_CODEC = codecFactory.jsonCodec(ExampleTable.class); @@ -55,7 +56,7 @@ private MetadataUtil() public static final class TestingTypeDeserializer extends FromStringDeserializer { - private final Map types = ImmutableMap.of( + private final Map types = ImmutableMap.of( StandardTypes.BOOLEAN, BOOLEAN, StandardTypes.BIGINT, BIGINT, StandardTypes.INTEGER, INTEGER, diff --git a/presto-example-http/src/test/java/com/facebook/presto/example/TestExampleRecordSet.java b/presto-example-http/src/test/java/com/facebook/presto/example/TestExampleRecordSet.java index a99c92d97bc8..f7dcdebff381 100644 --- a/presto-example-http/src/test/java/com/facebook/presto/example/TestExampleRecordSet.java +++ b/presto-example-http/src/test/java/com/facebook/presto/example/TestExampleRecordSet.java @@ -55,7 +55,7 @@ public void testGetColumnTypes() new ExampleColumnHandle("test", "text", createUnboundedVarcharType(), 0))); assertEquals(recordSet.getColumnTypes(), ImmutableList.of(BIGINT, BIGINT, createUnboundedVarcharType())); - recordSet = new ExampleRecordSet(new ExampleSplit("test", "schema", "table", dataUri), ImmutableList.of()); + recordSet = new ExampleRecordSet(new ExampleSplit("test", "schema", "table", dataUri), ImmutableList.of()); assertEquals(recordSet.getColumnTypes(), ImmutableList.of()); } 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 b4aec1598d02..576e93776daf 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 @@ -88,7 +88,7 @@ public class BackgroundHiveSplitLoader private final String connectorId; private final Table table; private final Optional bucketHandle; - private final List buckets; + private final Optional bucket; private final HdfsEnvironment hdfsEnvironment; private final NamenodeStats namenodeStats; private final DirectoryLister directoryLister; @@ -123,7 +123,7 @@ public BackgroundHiveSplitLoader( Table table, Iterable partitions, Optional bucketHandle, - List buckets, + Optional bucket, ConnectorSession session, HdfsEnvironment hdfsEnvironment, NamenodeStats namenodeStats, @@ -136,7 +136,7 @@ public BackgroundHiveSplitLoader( this.connectorId = connectorId; this.table = table; this.bucketHandle = bucketHandle; - this.buckets = buckets; + this.bucket = bucket; this.maxSplitSize = getMaxSplitSize(session); this.maxPartitionBatchSize = maxPartitionBatchSize; this.session = session; @@ -358,30 +358,25 @@ private void loadPartition(HivePartitionMetadata partition) // If only one bucket could match: load that one file HiveFileIterator iterator = new HiveFileIterator(path, fs, directoryLister, namenodeStats, partitionName, inputFormat, schema, partitionKeys, effectivePredicate, partition.getColumnCoercions()); - if (!buckets.isEmpty()) { - int bucketCount = buckets.get(0).getBucketCount(); - List list = listAndSortBucketFiles(iterator, bucketCount); - - for (HiveBucket bucket : buckets) { - int bucketNumber = bucket.getBucketNumber(); - LocatedFileStatus file = list.get(bucketNumber); - boolean splittable = isSplittable(iterator.getInputFormat(), hdfsEnvironment.getFileSystem(session.getUser(), file.getPath()), file.getPath()); - - hiveSplitSource.addToQueue(createHiveSplits( - iterator.getPartitionName(), - file.getPath().toString(), - file.getBlockLocations(), - 0, - file.getLen(), - iterator.getSchema(), - iterator.getPartitionKeys(), - splittable, - session, - OptionalInt.of(bucketNumber), - effectivePredicate, - partition.getColumnCoercions())); - } + if (bucket.isPresent()) { + List locatedFileStatuses = listAndSortBucketFiles(iterator, bucket.get().getBucketCount()); + FileStatus file = locatedFileStatuses.get(bucket.get().getBucketNumber()); + BlockLocation[] blockLocations = fs.getFileBlockLocations(file, 0, file.getLen()); + boolean splittable = isSplittable(inputFormat, fs, file.getPath()); + hiveSplitSource.addToQueue(createHiveSplits( + partitionName, + file.getPath().toString(), + blockLocations, + 0, + file.getLen(), + schema, + partitionKeys, + splittable, + session, + OptionalInt.of(bucket.get().getBucketNumber()), + effectivePredicate, + partition.getColumnCoercions())); return; } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarBinaryHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarBinaryHiveRecordCursor.java index 5ec4a67985ab..0ed6b4fa6dd1 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarBinaryHiveRecordCursor.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/ColumnarBinaryHiveRecordCursor.java @@ -44,7 +44,6 @@ import java.util.Set; import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; -import static com.facebook.presto.hive.HiveErrorCode.HIVE_BAD_DATA; import static com.facebook.presto.hive.HiveErrorCode.HIVE_CURSOR_ERROR; import static com.facebook.presto.hive.HiveType.HIVE_BYTE; import static com.facebook.presto.hive.HiveType.HIVE_DATE; @@ -317,7 +316,7 @@ private void parseLongColumn(int column) bytes = fieldData.getData(); } catch (IOException e) { - throw new PrestoException(HIVE_BAD_DATA, e); + throw Throwables.propagate(e); } int start = fieldData.getStart(); 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 cbc3bb215409..05f7ad1d71e6 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 @@ -70,7 +70,6 @@ public class HdfsConfigurationUpdater private final boolean pinS3ClientToCurrentRegion; private final String s3UserAgentPrefix; private final HiveCompressionCodec compressionCodec; - private final int fileSystemMaxCacheSize; @Inject public HdfsConfigurationUpdater(HiveClientConfig hiveClientConfig) @@ -107,7 +106,6 @@ public HdfsConfigurationUpdater(HiveClientConfig hiveClientConfig) this.pinS3ClientToCurrentRegion = hiveClientConfig.isPinS3ClientToCurrentRegion(); this.s3UserAgentPrefix = hiveClientConfig.getS3UserAgentPrefix(); this.compressionCodec = hiveClientConfig.getHiveCompressionCodec(); - this.fileSystemMaxCacheSize = hiveClientConfig.getFileSystemMaxCacheSize(); } public void updateConfiguration(Configuration config) @@ -163,8 +161,6 @@ public void updateConfiguration(Configuration config) config.set(PrestoS3FileSystem.S3_SIGNER_TYPE, s3SignerType.getSignerType()); } - config.setInt("fs.cache.max-size", fileSystemMaxCacheSize); - configureCompression(config, compressionCodec); // set config for S3 diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveBucketing.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveBucketing.java index e3d75d05cd46..08277ac02d1b 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveBucketing.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveBucketing.java @@ -18,10 +18,7 @@ import com.facebook.presto.spi.Page; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.block.Block; -import com.facebook.presto.spi.predicate.Domain; import com.facebook.presto.spi.predicate.NullableValue; -import com.facebook.presto.spi.predicate.TupleDomain; -import com.facebook.presto.spi.predicate.ValueSet; import com.facebook.presto.spi.type.Type; import com.google.common.collect.ImmutableList; import com.google.common.primitives.Ints; @@ -49,7 +46,6 @@ import java.util.Set; import java.util.stream.Collectors; -import static com.facebook.presto.hive.HiveColumnHandle.BUCKET_COLUMN_NAME; import static com.facebook.presto.hive.HiveErrorCode.HIVE_INVALID_METADATA; import static com.facebook.presto.hive.HiveUtil.getRegularColumnHandles; import static com.facebook.presto.hive.HiveUtil.getTableStructFields; @@ -213,45 +209,9 @@ public static Optional getHiveBucketHandle(String connectorId, return Optional.of(new HiveBucketHandle(bucketColumns.build(), hiveBucketProperty.get().getBucketCount())); } - public static List getHiveBucketNumbers(Table table, TupleDomain effectivePredicate) + public static Optional getHiveBucket(Table table, Map bindings) { - if (!table.getStorage().getBucketProperty().isPresent()) { - return ImmutableList.of(); - } - - Optional> bindings = TupleDomain.extractFixedValues(effectivePredicate); - if (!bindings.isPresent()) { - return ImmutableList.of(); - } - Optional singleBucket = getHiveBucket(table, bindings.get()); - if (singleBucket.isPresent()) { - return ImmutableList.of(singleBucket.get()); - } - - if (!effectivePredicate.getDomains().isPresent()) { - return ImmutableList.of(); - } - Optional domain = effectivePredicate.getDomains().get().entrySet().stream() - .filter(entry -> ((HiveColumnHandle) entry.getKey()).getName().equals(BUCKET_COLUMN_NAME)) - .findFirst() - .map(Entry::getValue); - if (!domain.isPresent()) { - return ImmutableList.of(); - } - ValueSet values = domain.get().getValues(); - ImmutableList.Builder builder = ImmutableList.builder(); - int bucketCount = table.getStorage().getBucketProperty().get().getBucketCount(); - for (int i = 0; i < bucketCount; i++) { - if (values.containsValue((long) i)) { - builder.add(new HiveBucket(i, bucketCount)); - } - } - return builder.build(); - } - - private static Optional getHiveBucket(Table table, Map bindings) - { - if (bindings.isEmpty()) { + if (!table.getStorage().getBucketProperty().isPresent() || bindings.isEmpty()) { return Optional.empty(); } 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 5700d19a98f2..9606972aedfc 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 @@ -49,7 +49,6 @@ public class HiveClientConfig private String timeZone = TimeZone.getDefault().getID(); private DataSize maxSplitSize = new DataSize(64, MEGABYTE); - private int maxPartitionsPerScan = 100_000; private int maxOutstandingSplits = 1_000; private int maxSplitIteratorThreads = 1_000; private int minPartitionBatchSize = 10; @@ -68,8 +67,6 @@ public class HiveClientConfig private Duration metastoreCacheTtl = new Duration(1, TimeUnit.HOURS); private Duration metastoreRefreshInterval = new Duration(1, TimeUnit.SECONDS); - private long metastoreCacheMaximumSize = 10000; - private long perTransactionMetastoreCacheMaximumSize = 1000; private int maxMetastoreRefreshThreads = 100; private HostAndPort metastoreSocksProxy; private Duration metastoreTimeout = new Duration(10, TimeUnit.SECONDS); @@ -141,8 +138,6 @@ public class HiveClientConfig private boolean bucketExecutionEnabled = true; private boolean bucketWritingEnabled = true; - private int fileSystemMaxCacheSize = 1000; - public int getMaxInitialSplits() { return maxInitialSplits; @@ -252,20 +247,6 @@ public HiveClientConfig setMaxSplitSize(DataSize maxSplitSize) return this; } - @Min(1) - public int getMaxPartitionsPerScan() - { - return maxPartitionsPerScan; - } - - @Config("hive.max-partitions-per-scan") - @ConfigDescription("Maximum allowed partitions for a single table scan") - public HiveClientConfig setMaxPartitionsPerScan(int maxPartitionsPerScan) - { - this.maxPartitionsPerScan = maxPartitionsPerScan; - return this; - } - @Min(1) public int getMaxOutstandingSplits() { @@ -348,32 +329,6 @@ public HiveClientConfig setMetastoreRefreshInterval(Duration metastoreRefreshInt return this; } - public long getMetastoreCacheMaximumSize() - { - return metastoreCacheMaximumSize; - } - - @Min(1) - @Config("hive.metastore-cache-maximum-size") - public HiveClientConfig setMetastoreCacheMaximumSize(long metastoreCacheMaximumSize) - { - this.metastoreCacheMaximumSize = metastoreCacheMaximumSize; - return this; - } - - public long getPerTransactionMetastoreCacheMaximumSize() - { - return perTransactionMetastoreCacheMaximumSize; - } - - @Min(1) - @Config("hive.per-transaction-metastore-cache-maximum-size") - public HiveClientConfig setPerTransactionMetastoreCacheMaximumSize(long perTransactionMetastoreCacheMaximumSize) - { - this.perTransactionMetastoreCacheMaximumSize = perTransactionMetastoreCacheMaximumSize; - return this; - } - @Min(1) public int getMaxMetastoreRefreshThreads() { @@ -1143,7 +1098,7 @@ public boolean isBucketExecutionEnabled() } @Config("hive.bucket-execution") - @ConfigDescription("Enable bucket-aware execution: only use a single worker per bucket") + @ConfigDescription("Use bucketing to speed up execution") public HiveClientConfig setBucketExecutionEnabled(boolean bucketExecutionEnabled) { this.bucketExecutionEnabled = bucketExecutionEnabled; @@ -1162,17 +1117,4 @@ public HiveClientConfig setBucketWritingEnabled(boolean bucketWritingEnabled) this.bucketWritingEnabled = bucketWritingEnabled; return this; } - - public int getFileSystemMaxCacheSize() - { - return fileSystemMaxCacheSize; - } - - @Config("hive.fs.cache.max-size") - @ConfigDescription("Hadoop FileSystem cache size") - public HiveClientConfig setFileSystemMaxCacheSize(int fileSystemMaxCacheSize) - { - this.fileSystemMaxCacheSize = fileSystemMaxCacheSize; - return this; - } } 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 d5bfabc40d90..7a187f48015c 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 @@ -92,8 +92,6 @@ public void configure(Binder binder) binder.bind(HiveMetastore.class).to(ThriftHiveMetastore.class).in(Scopes.SINGLETON); binder.bind(ExtendedHiveMetastore.class).annotatedWith(ForCachingHiveMetastore.class).to(BridgingHiveMetastore.class).in(Scopes.SINGLETON); binder.bind(ExtendedHiveMetastore.class).to(CachingHiveMetastore.class).in(Scopes.SINGLETON); - newExporter(binder).export(HiveMetastore.class) - .as(generatedNameOf(ThriftHiveMetastore.class, connectorId)); newExporter(binder).export(ExtendedHiveMetastore.class) .as(generatedNameOf(CachingHiveMetastore.class, connectorId)); } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveColumnHandle.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveColumnHandle.java index 47e36d293ad8..d007483ba8a6 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveColumnHandle.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveColumnHandle.java @@ -24,7 +24,6 @@ import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.HIDDEN; import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; -import static com.facebook.presto.hive.HiveType.HIVE_INT; import static com.facebook.presto.hive.HiveType.HIVE_LONG; import static com.facebook.presto.hive.HiveType.HIVE_STRING; import static com.facebook.presto.hive.util.Types.checkType; @@ -41,11 +40,6 @@ public class HiveColumnHandle public static final HiveType PATH_HIVE_TYPE = HIVE_STRING; public static final TypeSignature PATH_TYPE_SIGNATURE = PATH_HIVE_TYPE.getTypeSignature(); - public static final int BUCKET_COLUMN_INDEX = -12; - public static final String BUCKET_COLUMN_NAME = "$bucket"; - public static final HiveType BUCKET_HIVE_TYPE = HIVE_INT; - public static final TypeSignature BUCKET_TYPE_SIGNATURE = BUCKET_HIVE_TYPE.getTypeSignature(); - private static final String UPDATE_ROW_ID_COLUMN_NAME = "$shard_row_id"; public enum ColumnType @@ -187,18 +181,8 @@ public static HiveColumnHandle pathColumnHandle(String connectorId) return new HiveColumnHandle(connectorId, PATH_COLUMN_NAME, PATH_HIVE_TYPE, PATH_TYPE_SIGNATURE, PATH_COLUMN_INDEX, HIDDEN); } - public static HiveColumnHandle bucketColumnHandle(String connectorId) - { - return new HiveColumnHandle(connectorId, BUCKET_COLUMN_NAME, BUCKET_HIVE_TYPE, BUCKET_TYPE_SIGNATURE, BUCKET_COLUMN_INDEX, HIDDEN); - } - public static boolean isPathColumnHandle(HiveColumnHandle column) { return column.getHiveColumnIndex() == PATH_COLUMN_INDEX; } - - public static boolean isBucketColumnHandle(HiveColumnHandle column) - { - return column.getHiveColumnIndex() == BUCKET_COLUMN_INDEX; - } } 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 2ca5348991b3..70307a596afd 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 @@ -18,7 +18,6 @@ import com.facebook.presto.spi.ErrorType; import static com.facebook.presto.spi.ErrorType.EXTERNAL; -import static com.facebook.presto.spi.ErrorType.USER_ERROR; public enum HiveErrorCode implements ErrorCodeSupplier @@ -51,8 +50,7 @@ public enum HiveErrorCode HIVE_WRITER_OPEN_ERROR(25, EXTERNAL), HIVE_WRITER_CLOSE_ERROR(26, EXTERNAL), HIVE_WRITER_DATA_ERROR(27, EXTERNAL), - HIVE_INVALID_BUCKET_FILES(28, EXTERNAL), - HIVE_EXCEEDED_PARTITION_LIMIT(29, USER_ERROR); + HIVE_INVALID_BUCKET_FILES(28, EXTERNAL); private final ErrorCode errorCode; 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 01dc1a1a3655..ab86a5d5f044 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 @@ -40,11 +40,9 @@ import com.facebook.presto.spi.TableNotFoundException; import com.facebook.presto.spi.ViewNotFoundException; import com.facebook.presto.spi.connector.ConnectorMetadata; -import com.facebook.presto.spi.predicate.Domain; import com.facebook.presto.spi.predicate.NullableValue; import com.facebook.presto.spi.predicate.TupleDomain; import com.facebook.presto.spi.security.Privilege; -import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.TypeManager; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Verify; @@ -68,7 +66,6 @@ import java.io.IOException; import java.util.Collection; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -79,7 +76,6 @@ import java.util.function.Predicate; import java.util.stream.Collectors; -import static com.facebook.presto.hive.HiveColumnHandle.BUCKET_COLUMN_NAME; import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.HIDDEN; import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.PARTITION_KEY; import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; @@ -89,10 +85,8 @@ import static com.facebook.presto.hive.HiveErrorCode.HIVE_CONCURRENT_MODIFICATION_DETECTED; import static com.facebook.presto.hive.HiveErrorCode.HIVE_INVALID_METADATA; import static com.facebook.presto.hive.HiveErrorCode.HIVE_TIMEZONE_MISMATCH; -import static com.facebook.presto.hive.HiveErrorCode.HIVE_UNKNOWN_ERROR; import static com.facebook.presto.hive.HiveErrorCode.HIVE_UNSUPPORTED_FORMAT; import static com.facebook.presto.hive.HiveErrorCode.HIVE_WRITER_CLOSE_ERROR; -import static com.facebook.presto.hive.HiveSessionProperties.isBucketExecutionEnabled; import static com.facebook.presto.hive.HiveTableProperties.BUCKETED_BY_PROPERTY; import static com.facebook.presto.hive.HiveTableProperties.BUCKET_COUNT_PROPERTY; import static com.facebook.presto.hive.HiveTableProperties.EXTERNAL_LOCATION_PROPERTY; @@ -105,7 +99,7 @@ import static com.facebook.presto.hive.HiveType.HIVE_STRING; import static com.facebook.presto.hive.HiveType.toHiveType; import static com.facebook.presto.hive.HiveUtil.PRESTO_VIEW_FLAG; -import static com.facebook.presto.hive.HiveUtil.columnExtraInfo; +import static com.facebook.presto.hive.HiveUtil.annotateColumnComment; import static com.facebook.presto.hive.HiveUtil.decodeViewData; import static com.facebook.presto.hive.HiveUtil.encodeViewData; import static com.facebook.presto.hive.HiveUtil.hiveColumnHandles; @@ -125,7 +119,6 @@ import static com.facebook.presto.spi.StandardErrorCode.INVALID_TABLE_PROPERTY; import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; import static com.facebook.presto.spi.StandardErrorCode.SCHEMA_NOT_EMPTY; -import static com.facebook.presto.spi.predicate.TupleDomain.withColumnDomains; import static com.google.common.base.MoreObjects.toStringHelper; import static com.google.common.base.Verify.verify; import static com.google.common.collect.Iterables.concat; @@ -155,6 +148,7 @@ public class HiveMetadata private final TableParameterCodec tableParameterCodec; private final JsonCodec partitionUpdateCodec; private final boolean respectTableFormat; + private final boolean bucketExecutionEnabled; private final boolean bucketWritingEnabled; private final HiveStorageFormat defaultStorageFormat; private final TypeTranslator typeTranslator; @@ -168,6 +162,7 @@ public HiveMetadata( DateTimeZone timeZone, boolean allowCorruptWritesForTesting, boolean respectTableFormat, + boolean bucketExecutionEnabled, boolean bucketWritingEnabled, HiveStorageFormat defaultStorageFormat, TypeManager typeManager, @@ -190,6 +185,7 @@ public HiveMetadata( this.tableParameterCodec = requireNonNull(tableParameterCodec, "tableParameterCodec is null"); this.partitionUpdateCodec = requireNonNull(partitionUpdateCodec, "partitionUpdateCodec is null"); this.respectTableFormat = respectTableFormat; + this.bucketExecutionEnabled = bucketExecutionEnabled; this.bucketWritingEnabled = bucketWritingEnabled; this.defaultStorageFormat = requireNonNull(defaultStorageFormat, "defaultStorageFormat is null"); this.typeTranslator = requireNonNull(typeTranslator, "typeTranslator is null"); @@ -269,7 +265,7 @@ private ConnectorTableMetadata getTableMetadata(SchemaTableName tableName) public Optional getInfo(ConnectorTableLayoutHandle layoutHandle) { HiveTableLayoutHandle tableLayoutHandle = checkType(layoutHandle, HiveTableLayoutHandle.class, "layoutHandle"); - if (tableLayoutHandle.getPartitions().isPresent()) { + if (!tableLayoutHandle.getPartitions().isPresent()) { return Optional.of(new HiveInputInfo(tableLayoutHandle.getPartitions().get().stream() .map(HivePartition::getPartitionId) .collect(Collectors.toList()))); @@ -486,16 +482,6 @@ private static Table buildTableObject( } } - ImmutableMap.Builder tableParameters = ImmutableMap.builder() - .put("comment", "Created by Presto") - .put(PRESTO_VERSION_NAME, prestoVersion) - .put(PRESTO_QUERY_ID_NAME, queryId) - .putAll(additionalTableParameters); - - if (external) { - tableParameters.put("EXTERNAL", "TRUE"); - } - Table.Builder tableBuilder = Table.builder() .setDatabaseName(schemaName) .setTableName(tableName) @@ -503,8 +489,12 @@ private static Table buildTableObject( .setTableType((external ? EXTERNAL_TABLE : MANAGED_TABLE).name()) .setDataColumns(columns.build()) .setPartitionColumns(partitionColumns) - .setParameters(tableParameters.build()); - + .setParameters(ImmutableMap.builder() + .put("comment", "Created by Presto") + .put(PRESTO_VERSION_NAME, prestoVersion) + .put(PRESTO_QUERY_ID_NAME, queryId) + .putAll(additionalTableParameters) + .build()); tableBuilder.getStorageBuilder() .setStorageFormat(fromHiveStorageFormat(hiveStorageFormat)) .setBucketProperty(bucketProperty) @@ -1048,7 +1038,11 @@ public List getTableLayouts(ConnectorSession session { HiveTableHandle handle = checkType(tableHandle, HiveTableHandle.class, "tableHandle"); - HivePartitionResult hivePartitionResult = partitionManager.getPartitions(metastore, tableHandle, constraint); + HivePartitionResult hivePartitionResult = partitionManager.getPartitions(session, metastore, tableHandle, constraint.getSummary()); + + List partitions = hivePartitionResult.getPartitions().stream() + .filter(partition -> constraint.predicate().test(partition.getKeys())) + .collect(toList()); return ImmutableList.of(new ConnectorTableLayoutResult( getTableLayout( @@ -1056,7 +1050,7 @@ public List getTableLayouts(ConnectorSession session new HiveTableLayoutHandle( handle.getClientId(), ImmutableList.copyOf(hivePartitionResult.getPartitionColumns()), - hivePartitionResult.getPartitions(), + partitions, hivePartitionResult.getEnforcedConstraint(), hivePartitionResult.getBucketHandle())), hivePartitionResult.getUnenforcedConstraint())); @@ -1067,21 +1061,22 @@ public ConnectorTableLayout getTableLayout(ConnectorSession session, ConnectorTa { HiveTableLayoutHandle hiveLayoutHandle = checkType(layoutHandle, HiveTableLayoutHandle.class, "layoutHandle"); List partitionColumns = hiveLayoutHandle.getPartitionColumns(); - List partitions = hiveLayoutHandle.getPartitions().get(); + List> partitionDomains = hiveLayoutHandle.getPartitions().get().stream() + .map(HivePartition::getTupleDomain) + .collect(toList()); - TupleDomain predicate = createPredicate(partitionColumns, partitions); + TupleDomain predicate = TupleDomain.none(); + if (!partitionDomains.isEmpty()) { + predicate = TupleDomain.columnWiseUnion(partitionDomains); + } Optional discretePredicates = Optional.empty(); if (!partitionColumns.isEmpty()) { - // Do not create tuple domains for every partition at the same time! - // There can be a huge number of partitions so use an iterable so - // all domains do not need to be in memory at the same time. - Iterable> partitionDomains = Iterables.transform(partitions, (hivePartition) -> TupleDomain.fromFixedValues(hivePartition.getKeys())); discretePredicates = Optional.of(new DiscretePredicates(partitionColumns, partitionDomains)); } Optional nodePartitioning = Optional.empty(); - if (isBucketExecutionEnabled(session) && hiveLayoutHandle.getBucketHandle().isPresent()) { + if (bucketExecutionEnabled && hiveLayoutHandle.getBucketHandle().isPresent()) { nodePartitioning = hiveLayoutHandle.getBucketHandle().map(hiveBucketHandle -> new ConnectorNodePartitioning( new HivePartitioningHandle( connectorId, @@ -1104,34 +1099,10 @@ public ConnectorTableLayout getTableLayout(ConnectorSession session, ConnectorTa ImmutableList.of()); } - private static TupleDomain createPredicate(List partitionColumns, List partitions) - { - if (partitions.isEmpty()) { - return TupleDomain.none(); - } - - Map domains = new HashMap<>(partitionColumns.size()); - - for (HivePartition partition : partitions) { - Map partitionValues = partition.getKeys(); - for (ColumnHandle column : partitionColumns) { - NullableValue nullableValue = partitionValues.get(column); - if (nullableValue == null) { - throw new PrestoException(HIVE_UNKNOWN_ERROR, format("Partition %s does now have a value for partition column %s", partition, column)); - } - - Type type = nullableValue.getType(); - Domain domain = nullableValue.isNull() ? Domain.onlyNull(type) : Domain.singleValue(type, nullableValue.getValue()); - domains.merge(column, domain, Domain::union); - } - } - return withColumnDomains(domains); - } - @Override public Optional getInsertLayout(ConnectorSession session, ConnectorTableHandle tableHandle) { - HivePartitionResult hivePartitionResult = partitionManager.getPartitions(metastore, tableHandle, Constraint.alwaysTrue()); + HivePartitionResult hivePartitionResult = partitionManager.getPartitions(session, metastore, tableHandle, TupleDomain.all()); if (!hivePartitionResult.getBucketHandle().isPresent()) { return Optional.empty(); } @@ -1319,19 +1290,15 @@ private static Function columnMetadataGetter(T builder.put(field.getName(), Optional.empty()); } } - // add hidden columns + // add hidden column builder.put(PATH_COLUMN_NAME, Optional.empty()); - if (table.getStorage().getBucketProperty().isPresent()) { - builder.put(BUCKET_COLUMN_NAME, Optional.empty()); - } Map> columnComment = builder.build(); return handle -> new ColumnMetadata( handle.getName(), typeManager.getType(handle.getTypeSignature()), - columnComment.get(handle.getName()).orElse(null), - columnExtraInfo(handle.isPartitionKey()), + annotateColumnComment(columnComment.get(handle.getName()), handle.isPartitionKey()), handle.isHidden()); } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadataFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadataFactory.java index e6641fb64d86..607375dac98c 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadataFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveMetadataFactory.java @@ -35,10 +35,10 @@ public class HiveMetadataFactory private final String connectorId; private final boolean allowCorruptWritesForTesting; private final boolean respectTableFormat; + private final boolean bucketExecutionEnabled; private final boolean bucketWritingEnabled; private final boolean skipDeletionForAlter; private final HiveStorageFormat defaultStorageFormat; - private final long perTransactionCacheMaximumSize; private final ExtendedHiveMetastore metastore; private final HdfsEnvironment hdfsEnvironment; private final HivePartitionManager partitionManager; @@ -76,9 +76,9 @@ public HiveMetadataFactory( hiveClientConfig.getAllowCorruptWritesForTesting(), hiveClientConfig.isRespectTableFormat(), hiveClientConfig.isSkipDeletionForAlter(), + hiveClientConfig.isBucketExecutionEnabled(), hiveClientConfig.isBucketWritingEnabled(), hiveClientConfig.getHiveStorageFormat(), - hiveClientConfig.getPerTransactionMetastoreCacheMaximumSize(), typeManager, locationService, tableParameterCodec, @@ -98,9 +98,9 @@ public HiveMetadataFactory( boolean allowCorruptWritesForTesting, boolean respectTableFormat, boolean skipDeletionForAlter, + boolean bucketExecutionEnabled, boolean bucketWritingEnabled, HiveStorageFormat defaultStorageFormat, - long perTransactionCacheMaximumSize, TypeManager typeManager, LocationService locationService, TableParameterCodec tableParameterCodec, @@ -114,9 +114,9 @@ public HiveMetadataFactory( this.allowCorruptWritesForTesting = allowCorruptWritesForTesting; this.respectTableFormat = respectTableFormat; this.skipDeletionForAlter = skipDeletionForAlter; + this.bucketExecutionEnabled = bucketExecutionEnabled; this.bucketWritingEnabled = bucketWritingEnabled; this.defaultStorageFormat = requireNonNull(defaultStorageFormat, "defaultStorageFormat is null"); - this.perTransactionCacheMaximumSize = perTransactionCacheMaximumSize; this.metastore = requireNonNull(metastore, "metastore is null"); this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); @@ -145,7 +145,7 @@ public HiveMetadata create() connectorId, new SemiTransactionalHiveMetastore( hdfsEnvironment, - CachingHiveMetastore.memoizeMetastore(metastore, perTransactionCacheMaximumSize), // per-transaction cache + CachingHiveMetastore.memoizeMetastore(metastore), // per-transaction cache renameExecution, skipDeletionForAlter), hdfsEnvironment, @@ -153,6 +153,7 @@ public HiveMetadata create() timeZone, allowCorruptWritesForTesting, respectTableFormat, + bucketExecutionEnabled, bucketWritingEnabled, defaultStorageFormat, typeManager, 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 9b262cab62d6..1234aa5ea3a8 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 @@ -39,7 +39,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.OptionalInt; import java.util.Properties; import java.util.Set; @@ -114,7 +113,6 @@ private ConnectorPageSource doCreatePageSource(ConnectorSession session, Connect hdfsEnvironment.getConfiguration(path), session, path, - hiveSplit.getBucketNumber(), hiveSplit.getStart(), hiveSplit.getLength(), hiveSplit.getSchema(), @@ -137,7 +135,6 @@ public static Optional createHivePageSource( Configuration configuration, ConnectorSession session, Path path, - OptionalInt bucketNumber, long start, long length, Properties schema, @@ -148,7 +145,7 @@ public static Optional createHivePageSource( TypeManager typeManager, Map columnCoercions) { - List columnMappings = ColumnMapping.buildColumnMappings(partitionKeys, hiveColumns, columnCoercions, path, bucketNumber); + List columnMappings = ColumnMapping.buildColumnMappings(partitionKeys, hiveColumns, columnCoercions, path); List regularColumnMappings = ColumnMapping.extractRegularColumnMappings(columnMappings); for (HivePageSourceFactory pageSourceFactory : pageSourceFactories) { @@ -273,8 +270,7 @@ public static List buildColumnMappings( List partitionKeys, List columns, Map columnCoercions, - Path path, - OptionalInt bucketNumber) + Path path) { Map partitionKeysByName = uniqueIndex(partitionKeys, HivePartitionKey::getName); int regularIndex = 0; @@ -292,7 +288,7 @@ public static List buildColumnMappings( // prepare the prefilled value HivePartitionKey partitionKey = partitionKeysByName.get(column.getName()); - prefilledValue = getPrefilledColumnValue(column, partitionKey, path, bucketNumber); + prefilledValue = getPrefilledColumnValue(column, partitionKey, path); } Optional coercionFrom = Optional.ofNullable(columnCoercions.get(column.getHiveColumnIndex())); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePartition.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePartition.java index f4998f15a70d..701929c71139 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePartition.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePartition.java @@ -19,9 +19,9 @@ import com.facebook.presto.spi.predicate.TupleDomain; import com.google.common.collect.ImmutableMap; -import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; import static com.facebook.presto.hive.HiveBucketing.HiveBucket; import static java.util.Objects.requireNonNull; @@ -34,24 +34,33 @@ public class HivePartition private final TupleDomain effectivePredicate; private final String partitionId; private final Map keys; - private final List buckets; + private final Optional bucket; - public HivePartition(SchemaTableName tableName, TupleDomain effectivePredicate, List buckets) + public HivePartition(SchemaTableName tableName, TupleDomain effectivePredicate) { - this(tableName, effectivePredicate, UNPARTITIONED_ID, ImmutableMap.of(), buckets); + this.tableName = requireNonNull(tableName, "tableName is null"); + this.effectivePredicate = requireNonNull(effectivePredicate, "effectivePredicate is null"); + this.partitionId = UNPARTITIONED_ID; + this.keys = ImmutableMap.of(); + this.bucket = Optional.empty(); + } + + public HivePartition(SchemaTableName tableName, TupleDomain effectivePredicate, Optional bucket) + { + this(tableName, effectivePredicate, UNPARTITIONED_ID, ImmutableMap.of(), bucket); } public HivePartition(SchemaTableName tableName, TupleDomain effectivePredicate, String partitionId, Map keys, - List buckets) + Optional bucket) { this.tableName = requireNonNull(tableName, "tableName is null"); this.effectivePredicate = requireNonNull(effectivePredicate, "effectivePredicate is null"); this.partitionId = requireNonNull(partitionId, "partitionId is null"); this.keys = ImmutableMap.copyOf(requireNonNull(keys, "keys is null")); - this.buckets = requireNonNull(buckets, "bucket number is null"); + this.bucket = requireNonNull(bucket, "bucket number is null"); } public SchemaTableName getTableName() @@ -69,14 +78,19 @@ public String getPartitionId() return partitionId; } + public TupleDomain getTupleDomain() + { + return TupleDomain.fromFixedValues(keys); + } + public Map getKeys() { return keys; } - public List getBuckets() + public Optional getBucket() { - return buckets; + return bucket; } @Override diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java index 067c66093199..4185f5ea9ad8 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePartitionManager.java @@ -16,8 +16,8 @@ import com.facebook.presto.hive.metastore.SemiTransactionalHiveMetastore; import com.facebook.presto.hive.metastore.Table; import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.ConnectorTableHandle; -import com.facebook.presto.spi.Constraint; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.SchemaTableName; import com.facebook.presto.spi.TableNotFoundException; @@ -25,7 +25,6 @@ import com.facebook.presto.spi.predicate.NullableValue; import com.facebook.presto.spi.predicate.TupleDomain; import com.facebook.presto.spi.predicate.ValueSet; -import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.TypeManager; import com.google.common.base.Predicates; import com.google.common.collect.ImmutableList; @@ -43,10 +42,8 @@ import java.util.Map; import java.util.Optional; -import static com.facebook.presto.hive.HiveBucketing.HiveBucket; +import static com.facebook.presto.hive.HiveBucketing.getHiveBucket; import static com.facebook.presto.hive.HiveBucketing.getHiveBucketHandle; -import static com.facebook.presto.hive.HiveBucketing.getHiveBucketNumbers; -import static com.facebook.presto.hive.HiveErrorCode.HIVE_EXCEEDED_PARTITION_LIMIT; import static com.facebook.presto.hive.HiveUtil.getPartitionKeyColumnHandles; import static com.facebook.presto.hive.HiveUtil.parsePartitionValue; import static com.facebook.presto.hive.util.Types.checkType; @@ -56,7 +53,6 @@ import static com.google.common.base.Strings.isNullOrEmpty; import static java.lang.String.format; import static java.util.Objects.requireNonNull; -import static java.util.stream.Collectors.toList; import static org.apache.hadoop.hive.metastore.ProtectMode.getProtectModeFromString; public class HivePartitionManager @@ -67,7 +63,6 @@ public class HivePartitionManager private final String connectorId; private final DateTimeZone timeZone; private final boolean assumeCanonicalPartitionKeys; - private final int maxPartitions; private final int domainCompactionThreshold; private final TypeManager typeManager; @@ -80,8 +75,8 @@ public HivePartitionManager( this(connectorId, typeManager, hiveClientConfig.getDateTimeZone(), + hiveClientConfig.getMaxOutstandingSplits(), hiveClientConfig.isAssumeCanonicalPartitionKeys(), - hiveClientConfig.getMaxPartitionsPerScan(), hiveClientConfig.getDomainCompactionThreshold()); } @@ -89,31 +84,31 @@ public HivePartitionManager( HiveConnectorId connectorId, TypeManager typeManager, DateTimeZone timeZone, + int maxOutstandingSplits, boolean assumeCanonicalPartitionKeys, - int maxPartitions, int domainCompactionThreshold) { this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); this.timeZone = requireNonNull(timeZone, "timeZone is null"); + checkArgument(maxOutstandingSplits >= 1, "maxOutstandingSplits must be at least 1"); this.assumeCanonicalPartitionKeys = assumeCanonicalPartitionKeys; - checkArgument(maxPartitions >= 1, "maxPartitions must be at least 1"); - this.maxPartitions = maxPartitions; checkArgument(domainCompactionThreshold >= 1, "domainCompactionThreshold must be at least 1"); this.domainCompactionThreshold = domainCompactionThreshold; this.typeManager = requireNonNull(typeManager, "typeManager is null"); } - public HivePartitionResult getPartitions(SemiTransactionalHiveMetastore metastore, ConnectorTableHandle tableHandle, Constraint constraint) + public HivePartitionResult getPartitions(ConnectorSession session, SemiTransactionalHiveMetastore metastore, ConnectorTableHandle tableHandle, TupleDomain effectivePredicate) { HiveTableHandle hiveTableHandle = checkType(tableHandle, HiveTableHandle.class, "tableHandle"); - TupleDomain effectivePredicate = constraint.getSummary(); + requireNonNull(effectivePredicate, "effectivePredicate is null"); SchemaTableName tableName = hiveTableHandle.getSchemaTableName(); Table table = getTable(metastore, tableName); Optional hiveBucketHandle = getHiveBucketHandle(connectorId, table); List partitionColumns = getPartitionKeyColumnHandles(connectorId, table); - List buckets = getHiveBucketNumbers(table, effectivePredicate); + Optional bucket = getHiveBucket(table, TupleDomain.extractFixedValues(effectivePredicate).get()); + TupleDomain compactEffectivePredicate = toCompactTupleDomain(effectivePredicate, domainCompactionThreshold); if (effectivePredicate.isNone()) { @@ -123,33 +118,21 @@ public HivePartitionResult getPartitions(SemiTransactionalHiveMetastore metastor if (partitionColumns.isEmpty()) { return new HivePartitionResult( partitionColumns, - ImmutableList.of(new HivePartition(tableName, compactEffectivePredicate, buckets)), + ImmutableList.of(new HivePartition(tableName, compactEffectivePredicate, bucket)), effectivePredicate, TupleDomain.none(), hiveBucketHandle); } - List partitionTypes = partitionColumns.stream() - .map(column -> typeManager.getType(column.getTypeSignature())) - .collect(toList()); - List partitionNames = getFilteredPartitionNames(metastore, tableName, partitionColumns, effectivePredicate); // do a final pass to filter based on fields that could not be used to filter the partitions - int partitionCount = 0; ImmutableList.Builder partitions = ImmutableList.builder(); for (String partitionName : partitionNames) { - Optional> values = parseValuesAndFilterPartition(partitionName, partitionColumns, partitionTypes, constraint); + Optional> values = parseValuesAndFilterPartition(partitionName, partitionColumns, effectivePredicate); if (values.isPresent()) { - if (partitionCount == maxPartitions) { - throw new PrestoException(HIVE_EXCEEDED_PARTITION_LIMIT, format( - "Query over table '%s' can potentially read more than %s partitions", - hiveTableHandle.getSchemaTableName(), - maxPartitions)); - } - partitionCount++; - partitions.add(new HivePartition(tableName, compactEffectivePredicate, partitionName, values.get(), buckets)); + partitions.add(new HivePartition(tableName, compactEffectivePredicate, partitionName, values.get(), bucket)); } } @@ -178,15 +161,17 @@ private static TupleDomain toCompactTupleDomain(TupleDomain> parseValuesAndFilterPartition(String partitionName, List partitionColumns, List partitionTypes, Constraint constraint) + private Optional> parseValuesAndFilterPartition(String partitionName, List partitionColumns, TupleDomain predicate) { + checkArgument(predicate.getDomains().isPresent()); + List partitionValues = extractPartitionKeyValues(partitionName); - Map domains = constraint.getSummary().getDomains().get(); + Map domains = predicate.getDomains().get(); ImmutableMap.Builder builder = ImmutableMap.builder(); for (int i = 0; i < partitionColumns.size(); i++) { HiveColumnHandle column = partitionColumns.get(i); - NullableValue parsedValue = parsePartitionValue(partitionName, partitionValues.get(i), partitionTypes.get(i), timeZone); + NullableValue parsedValue = parsePartitionValue(partitionName, partitionValues.get(i), typeManager.getType(column.getTypeSignature()), timeZone); Domain allowedDomain = domains.get(column); if (allowedDomain != null && !allowedDomain.includesNullableValue(parsedValue.getValue())) { @@ -194,13 +179,8 @@ private Optional> parseValuesAndFilterPartition } builder.put(column, parsedValue); } - Map values = builder.build(); - - if (!constraint.predicate().test(values)) { - return Optional.empty(); - } - return Optional.of(values); + return Optional.of(builder.build()); } private Table getTable(SemiTransactionalHiveMetastore metastore, SchemaTableName tableName) @@ -213,12 +193,12 @@ private Table getTable(SemiTransactionalHiveMetastore metastore, SchemaTableName String protectMode = table.getParameters().get(ProtectMode.PARAMETER_NAME); if (protectMode != null && getProtectModeFromString(protectMode).offline) { - throw new TableOfflineException(tableName, false, null); + throw new TableOfflineException(tableName); } String prestoOffline = table.getParameters().get(PRESTO_OFFLINE); if (!isNullOrEmpty(prestoOffline)) { - throw new TableOfflineException(tableName, true, prestoOffline); + throw new TableOfflineException(tableName, format("Table '%s' is offline for Presto: %s", tableName, prestoOffline)); } return table; 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 e81f9d7d5eb5..f5014d35b904 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 @@ -27,7 +27,6 @@ public final class HiveSessionProperties { - private static final String BUCKET_EXECUTION_ENABLED = "bucket_execution_enabled"; private static final String FORCE_LOCAL_SCHEDULING = "force_local_scheduling"; private static final String ORC_BLOOM_FILTERS_ENABLED = "orc_bloom_filters_enabled"; private static final String ORC_MAX_MERGE_DISTANCE = "orc_max_merge_distance"; @@ -46,11 +45,6 @@ public final class HiveSessionProperties public HiveSessionProperties(HiveClientConfig config) { sessionProperties = ImmutableList.of( - booleanSessionProperty( - BUCKET_EXECUTION_ENABLED, - "Enable bucket-aware execution: only use a single worker per bucket", - config.isBucketExecutionEnabled(), - false), booleanSessionProperty( FORCE_LOCAL_SCHEDULING, "Only schedule splits on workers colocated with data node", @@ -113,11 +107,6 @@ public List> getSessionProperties() return sessionProperties; } - public static boolean isBucketExecutionEnabled(ConnectorSession session) - { - return session.getProperty(BUCKET_EXECUTION_ENABLED, Boolean.class); - } - public static boolean isForceLocalScheduling(ConnectorSession session) { return session.getProperty(FORCE_LOCAL_SCHEDULING, Boolean.class); 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 129624a17257..49236ce9818b 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 @@ -151,7 +151,7 @@ public ConnectorSplitSource getSplits(ConnectorTransactionHandle transaction, Co return new FixedSplitSource(ImmutableList.of()); } SchemaTableName tableName = partition.getTableName(); - List buckets = partition.getBuckets(); + Optional bucket = partition.getBucket(); Optional bucketHandle = layout.getBucketHandle(); // sort partitions @@ -169,7 +169,7 @@ public ConnectorSplitSource getSplits(ConnectorTransactionHandle transaction, Co table.get(), hivePartitions, bucketHandle, - buckets, + bucket, session, hdfsEnvironment, namenodeStats, @@ -228,11 +228,11 @@ private Iterable getPartitionMetadata(SemiTransactionalHi String protectMode = partition.getParameters().get(ProtectMode.PARAMETER_NAME); String partName = makePartName(table.getPartitionColumns(), partition.getValues()); if (protectMode != null && getProtectModeFromString(protectMode).offline) { - throw new PartitionOfflineException(tableName, partName, false, null); + throw new PartitionOfflineException(tableName, partName); } String prestoOffline = partition.getParameters().get(PRESTO_OFFLINE); if (!isNullOrEmpty(prestoOffline)) { - throw new PartitionOfflineException(tableName, partName, true, prestoOffline); + throw new PartitionOfflineException(tableName, partName, format("Partition '%s' is offline for Presto: %s", partName, prestoOffline)); } // Verify that the partition schema matches the table schema. diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveType.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveType.java index 2a0059f8f291..acfef694ba55 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveType.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveType.java @@ -14,7 +14,6 @@ package com.facebook.presto.hive; import com.facebook.presto.spi.PrestoException; -import com.facebook.presto.spi.type.NamedTypeSignature; import com.facebook.presto.spi.type.StandardTypes; import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.TypeManager; @@ -35,7 +34,6 @@ import javax.annotation.Nonnull; import java.util.List; -import java.util.Locale; import static com.facebook.presto.hive.util.Types.checkType; import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; @@ -233,22 +231,11 @@ private static TypeSignature getTypeSignature(TypeInfo typeInfo) ImmutableList.of(TypeSignatureParameter.of(elementType))); case STRUCT: StructTypeInfo structTypeInfo = checkType(typeInfo, StructTypeInfo.class, "fieldInspector"); - List structFieldTypeInfos = structTypeInfo.getAllStructFieldTypeInfos(); - List structFieldNames = structTypeInfo.getAllStructFieldNames(); - if (structFieldTypeInfos.size() != structFieldNames.size()) { - throw new PrestoException(HiveErrorCode.HIVE_INVALID_METADATA, format("Invalid Hive struct type: %s", typeInfo)); - } - ImmutableList.Builder typeSignatureBuilder = ImmutableList.builder(); - for (int i = 0; i < structFieldTypeInfos.size(); i++) { - TypeSignature typeSignature = getTypeSignature(structFieldTypeInfos.get(i)); - // Lower case the struct field names. - // Otherwise, Presto will refuse to write to columns whose struct type has field names containing upper case characters. - // Users can't work around this by casting in their queries because Presto parser always lower case types. - // TODO: This is a hack. Presto engine should be able to handle identifiers in a case insensitive way where necessary. - String rowFieldName = structFieldNames.get(i).toLowerCase(Locale.US); - typeSignatureBuilder.add(TypeSignatureParameter.of(new NamedTypeSignature(rowFieldName, typeSignature))); - } - return new TypeSignature(StandardTypes.ROW, typeSignatureBuilder.build()); + List fieldTypes = structTypeInfo.getAllStructFieldTypeInfos() + .stream() + .map(HiveType::getTypeSignature) + .collect(toList()); + return new TypeSignature(StandardTypes.ROW, fieldTypes, structTypeInfo.getAllStructFieldNames()); } throw new PrestoException(NOT_SUPPORTED, format("Unsupported Hive type: %s", typeInfo)); } 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 c56b5cbd339f..476a77954e32 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 @@ -69,7 +69,6 @@ import java.util.Base64; import java.util.List; import java.util.Optional; -import java.util.OptionalInt; import java.util.Properties; import java.util.concurrent.TimeUnit; import java.util.regex.Matcher; @@ -77,8 +76,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.HiveColumnHandle.bucketColumnHandle; -import static com.facebook.presto.hive.HiveColumnHandle.isBucketColumnHandle; import static com.facebook.presto.hive.HiveColumnHandle.isPathColumnHandle; import static com.facebook.presto.hive.HiveColumnHandle.pathColumnHandle; import static com.facebook.presto.hive.HiveErrorCode.HIVE_CANNOT_OPEN_SPLIT; @@ -712,11 +709,8 @@ public static List hiveColumnHandles(String connectorId, Table // add the partition keys last (like Hive does) columns.addAll(getPartitionKeyColumnHandles(connectorId, table)); - // add hidden columns + // add hidden column columns.add(pathColumnHandle(connectorId)); - if (table.getStorage().getBucketProperty().isPresent()) { - columns.add(bucketColumnHandle(connectorId)); - } return columns.build(); } @@ -767,9 +761,18 @@ public static void checkCondition(boolean condition, ErrorCodeSupplier errorCode } @Nullable - public static String columnExtraInfo(boolean partitionKey) + public static String annotateColumnComment(Optional comment, boolean partitionKey) { - return partitionKey ? "partition key" : null; + String normalizedComment = comment.orElse("").trim(); + if (partitionKey) { + if (normalizedComment.isEmpty()) { + normalizedComment = "Partition Key"; + } + else { + normalizedComment = "Partition Key: " + normalizedComment; + } + } + return normalizedComment.isEmpty() ? null : normalizedComment; } public static List toPartitionValues(String partitionName) @@ -795,7 +798,7 @@ public static List toPartitionValues(String partitionName) return resultBuilder.build(); } - public static String getPrefilledColumnValue(HiveColumnHandle columnHandle, HivePartitionKey partitionKey, Path path, OptionalInt bucketNumber) + public static String getPrefilledColumnValue(HiveColumnHandle columnHandle, HivePartitionKey partitionKey, Path path) { if (partitionKey != null) { return partitionKey.getValue(); @@ -803,9 +806,6 @@ public static String getPrefilledColumnValue(HiveColumnHandle columnHandle, Hive if (isPathColumnHandle(columnHandle)) { return path.toString(); } - if (isBucketColumnHandle(columnHandle)) { - return String.valueOf(bucketNumber.getAsInt()); - } throw new PrestoException(NOT_SUPPORTED, "unsupported hidden column: " + columnHandle); } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java index cbd262f6e422..d57db778692c 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java @@ -114,7 +114,6 @@ import static java.util.UUID.randomUUID; import static java.util.stream.Collectors.toList; import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.COMPRESSRESULT; -import static org.apache.hadoop.hive.metastore.TableType.MANAGED_TABLE; import static org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory.getPrimitiveJavaObjectInspector; import static org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector; import static org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory.javaBooleanObjectInspector; @@ -333,10 +332,6 @@ public static Object getField(Type type, Block block, int position) public static void checkTableIsWritable(Table table) { - if (!table.getTableType().equals(MANAGED_TABLE.toString())) { - throw new PrestoException(NOT_SUPPORTED, "Cannot write to non-managed Hive table"); - } - checkWritable( new SchemaTableName(table.getDatabaseName(), table.getTableName()), Optional.empty(), @@ -369,18 +364,12 @@ private static void checkWritable( // verify online if (protectMode.offline) { - if (partitionName.isPresent()) { - throw new PartitionOfflineException(tableName, partitionName.get(), false, null); - } - throw new TableOfflineException(tableName, false, null); + throw new TableOfflineException(tableName, format("%s is offline", tablePartitionDescription)); } String prestoOffline = parameters.get(PRESTO_OFFLINE); if (!isNullOrEmpty(prestoOffline)) { - if (partitionName.isPresent()) { - throw new PartitionOfflineException(tableName, partitionName.get(), true, prestoOffline); - } - throw new TableOfflineException(tableName, true, prestoOffline); + throw new TableOfflineException(tableName, format("%s is offline for Presto: %s", tablePartitionDescription, prestoOffline)); } // verify not read only diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/PartitionOfflineException.java b/presto-hive/src/main/java/com/facebook/presto/hive/PartitionOfflineException.java index 35d1b56a779c..ca3e9915fdc3 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/PartitionOfflineException.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/PartitionOfflineException.java @@ -17,7 +17,6 @@ import com.facebook.presto.spi.SchemaTableName; import static com.facebook.presto.hive.HiveErrorCode.HIVE_PARTITION_OFFLINE; -import static com.google.common.base.Strings.isNullOrEmpty; import static java.util.Objects.requireNonNull; public class PartitionOfflineException @@ -26,11 +25,21 @@ public class PartitionOfflineException private final SchemaTableName tableName; private final String partition; - public PartitionOfflineException(SchemaTableName tableName, String partitionName, boolean forPresto, String offlineMessage) + public PartitionOfflineException(SchemaTableName tableName, String partition) { - super(HIVE_PARTITION_OFFLINE, formatMessage(tableName, partitionName, forPresto, offlineMessage)); + this(tableName, partition, String.format("Table '%s' partition '%s' is offline", tableName, partition)); + } + + public PartitionOfflineException(SchemaTableName tableName, + String partition, + String message) + { + super(HIVE_PARTITION_OFFLINE, message); + if (tableName == null) { + throw new NullPointerException("tableName is null"); + } this.tableName = requireNonNull(tableName, "tableName is null"); - this.partition = requireNonNull(partitionName, "partition is null"); + this.partition = requireNonNull(partition, "partition is null"); } public SchemaTableName getTableName() @@ -42,19 +51,4 @@ public String getPartition() { return partition; } - - private static String formatMessage(SchemaTableName tableName, String partitionName, boolean forPresto, String offlineMessage) - { - StringBuilder resultBuilder = new StringBuilder() - .append("Table '").append(tableName).append("'") - .append(" partition '").append(partitionName).append("'") - .append(" is offline"); - if (forPresto) { - resultBuilder.append(" for Presto"); - } - if (!isNullOrEmpty(offlineMessage)) { - resultBuilder.append(": ").append(offlineMessage); - } - return resultBuilder.toString(); - } } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/TableOfflineException.java b/presto-hive/src/main/java/com/facebook/presto/hive/TableOfflineException.java index b6188fda9789..e6421b0d33bc 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/TableOfflineException.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/TableOfflineException.java @@ -17,7 +17,6 @@ import com.facebook.presto.spi.SchemaTableName; import static com.facebook.presto.hive.HiveErrorCode.HIVE_TABLE_OFFLINE; -import static com.google.common.base.Strings.isNullOrEmpty; import static java.util.Objects.requireNonNull; public class TableOfflineException @@ -25,28 +24,19 @@ public class TableOfflineException { private final SchemaTableName tableName; - public TableOfflineException(SchemaTableName tableName, boolean forPresto, String offlineMessage) + public TableOfflineException(SchemaTableName tableName) { - super(HIVE_TABLE_OFFLINE, formatMessage(tableName, forPresto, offlineMessage)); - this.tableName = requireNonNull(tableName, "tableName is null"); + this(tableName, String.format("Table '%s' is offline", tableName)); } - public SchemaTableName getTableName() + public TableOfflineException(SchemaTableName tableName, String message) { - return tableName; + super(HIVE_TABLE_OFFLINE, message); + this.tableName = requireNonNull(tableName, "tableName is null"); } - private static String formatMessage(SchemaTableName tableName, boolean forPresto, String offlineMessage) + public SchemaTableName getTableName() { - StringBuilder resultBuilder = new StringBuilder() - .append("Table '").append(tableName).append("'") - .append(" is offline"); - if (forPresto) { - resultBuilder.append(" for Presto"); - } - if (!isNullOrEmpty(offlineMessage)) { - resultBuilder.append(": ").append(offlineMessage); - } - return resultBuilder.toString(); + return tableName; } } 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 bbb88ccb69d7..a917dd8f80a6 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 @@ -55,6 +55,11 @@ public BridgingHiveMetastore(HiveMetastore delegate) this.delegate = delegate; } + @Override + public void flushCache() + { + } + @Override public Optional getDatabase(String databaseName) { 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 f61360238488..f140efca9c89 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 @@ -79,33 +79,30 @@ public CachingHiveMetastore(@ForCachingHiveMetastore ExtendedHiveMetastore deleg this(requireNonNull(delegate, "delegate is null"), requireNonNull(executor, "executor is null"), requireNonNull(hiveClientConfig, "hiveClientConfig is null").getMetastoreCacheTtl(), - hiveClientConfig.getMetastoreRefreshInterval(), - hiveClientConfig.getMetastoreCacheMaximumSize()); + hiveClientConfig.getMetastoreRefreshInterval()); } - public CachingHiveMetastore(ExtendedHiveMetastore delegate, ExecutorService executor, Duration cacheTtl, Duration refreshInterval, long maximumSize) + public CachingHiveMetastore(ExtendedHiveMetastore delegate, ExecutorService executor, Duration cacheTtl, Duration refreshInterval) { this(requireNonNull(delegate, "delegate is null"), requireNonNull(executor, "executor is null"), OptionalLong.of(requireNonNull(cacheTtl, "cacheTtl is null").toMillis()), - OptionalLong.of(requireNonNull(refreshInterval, "refreshInterval is null").toMillis()), - maximumSize); + OptionalLong.of(requireNonNull(refreshInterval, "refreshInterval is null").toMillis())); } - public static CachingHiveMetastore memoizeMetastore(ExtendedHiveMetastore delegate, long maximumSize) + public static CachingHiveMetastore memoizeMetastore(ExtendedHiveMetastore delegate) { return new CachingHiveMetastore(requireNonNull(delegate, "delegate is null"), newDirectExecutorService(), OptionalLong.empty(), - OptionalLong.empty(), - maximumSize); + OptionalLong.empty()); } - private CachingHiveMetastore(ExtendedHiveMetastore delegate, ExecutorService executor, OptionalLong expiresAfterWriteMillis, OptionalLong refreshMills, long maximumSize) + private CachingHiveMetastore(ExtendedHiveMetastore delegate, ExecutorService executor, OptionalLong expiresAfterWriteMillis, OptionalLong refreshMills) { this.delegate = delegate; - databaseNamesCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize) + databaseNamesCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills) .build(asyncReloading(new CacheLoader>() { @Override @@ -116,7 +113,7 @@ public List load(String key) } }, executor)); - databaseCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize) + databaseCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills) .build(asyncReloading(new CacheLoader>() { @Override @@ -127,7 +124,7 @@ public Optional load(String databaseName) } }, executor)); - tableNamesCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize) + tableNamesCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills) .build(asyncReloading(new CacheLoader>>() { @Override @@ -138,7 +135,7 @@ public Optional> load(String databaseName) } }, executor)); - tableCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize) + tableCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills) .build(asyncReloading(new CacheLoader>() { @Override @@ -149,7 +146,7 @@ public Optional
load(HiveTableName hiveTableName) } }, executor)); - viewNamesCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize) + viewNamesCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills) .build(asyncReloading(new CacheLoader>>() { @Override @@ -160,7 +157,7 @@ public Optional> load(String databaseName) } }, executor)); - partitionNamesCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize) + partitionNamesCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills) .build(asyncReloading(new CacheLoader>>() { @Override @@ -171,7 +168,7 @@ public Optional> load(HiveTableName hiveTableName) } }, executor)); - partitionFilterCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize) + partitionFilterCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills) .build(asyncReloading(new CacheLoader>>() { @Override @@ -182,7 +179,7 @@ public Optional> load(PartitionFilter partitionFilter) } }, executor)); - partitionCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize) + partitionCache = newCacheBuilder(expiresAfterWriteMillis, refreshMills) .build(asyncReloading(new CacheLoader>() { @Override @@ -200,7 +197,7 @@ public Map> loadAll(Iterable>() { @Override @@ -211,7 +208,7 @@ public Set load(String user) } }, executor)); - userTablePrivileges = newCacheBuilder(expiresAfterWriteMillis, refreshMills, maximumSize) + userTablePrivileges = newCacheBuilder(expiresAfterWriteMillis, refreshMills) .build(asyncReloading(new CacheLoader>() { @Override @@ -223,6 +220,7 @@ public Set load(UserTableKey key) }, executor)); } + @Override @Managed public void flushCache() { @@ -612,7 +610,7 @@ public void revokeTablePrivileges(String databaseName, String tableName, String } } - private static CacheBuilder newCacheBuilder(OptionalLong expiresAfterWriteMillis, OptionalLong refreshMillis, long maximumSize) + private static CacheBuilder newCacheBuilder(OptionalLong expiresAfterWriteMillis, OptionalLong refreshMillis) { CacheBuilder cacheBuilder = CacheBuilder.newBuilder(); if (expiresAfterWriteMillis.isPresent()) { @@ -621,7 +619,6 @@ private static CacheBuilder newCacheBuilder(OptionalLong expires if (refreshMillis.isPresent()) { cacheBuilder = cacheBuilder.refreshAfterWrite(refreshMillis.getAsLong(), MILLISECONDS); } - cacheBuilder = cacheBuilder.maximumSize(maximumSize); return cacheBuilder; } 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 acecf54e0c49..ef621bbec02f 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 @@ -25,6 +25,8 @@ public interface ExtendedHiveMetastore { + void flushCache(); + Optional getDatabase(String databaseName); List getAllDatabases(); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/HiveMetastore.java b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/HiveMetastore.java index 5fe329f927bf..625992827784 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/HiveMetastore.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/HiveMetastore.java @@ -17,6 +17,7 @@ import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo; import org.apache.hadoop.hive.metastore.api.Table; +import org.weakref.jmx.Managed; import java.util.List; import java.util.Optional; @@ -41,6 +42,9 @@ public interface HiveMetastore void alterTable(String databaseName, String tableName, Table table); + @Managed + void flushCache(); + List getAllDatabases(); Optional> getAllTables(String databaseName); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/ThriftHiveMetastore.java b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/ThriftHiveMetastore.java index d0017a7b387a..e0798a402640 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/ThriftHiveMetastore.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/ThriftHiveMetastore.java @@ -83,21 +83,13 @@ public class ThriftHiveMetastore implements HiveMetastore { - private final ThriftHiveMetastoreStats stats; - private final HiveCluster clientProvider; - private final Function exceptionMapper; + private final ThriftHiveMetastoreStats stats = new ThriftHiveMetastoreStats(); + protected final HiveCluster clientProvider; @Inject public ThriftHiveMetastore(HiveCluster hiveCluster) - { - this(hiveCluster, new ThriftHiveMetastoreStats(), identity()); - } - - ThriftHiveMetastore(HiveCluster hiveCluster, ThriftHiveMetastoreStats stats, Function exceptionMapper) { this.clientProvider = requireNonNull(hiveCluster, "hiveCluster is null"); - this.stats = requireNonNull(stats, "stats is null"); - this.exceptionMapper = requireNonNull(exceptionMapper, "exceptionMapper is null"); } @Managed @@ -107,6 +99,12 @@ public ThriftHiveMetastoreStats getStats() return stats; } + @Override + @Managed + public void flushCache() + { + } + @Override public List getAllDatabases() { @@ -393,6 +391,9 @@ public void alterTable(String databaseName, String tableName, org.apache.hadoop. catch (NoSuchObjectException e) { throw new TableNotFoundException(new SchemaTableName(databaseName, tableName)); } + catch (InvalidOperationException | MetaException e) { + throw Throwables.propagate(e); + } catch (TException e) { throw new PrestoException(HIVE_METASTORE_ERROR, e); } @@ -810,10 +811,15 @@ private Set getPrivileges(String user, HiveObjectRef objectRe private RetryDriver retry() { return RetryDriver.retry() - .exceptionMapper(exceptionMapper) + .exceptionMapper(getExceptionMapper()) .stopOn(PrestoException.class); } + protected Function getExceptionMapper() + { + return identity(); + } + private RuntimeException propagate(Throwable throwable) { if (throwable instanceof InterruptedException) { diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSource.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSource.java index 7dcb4b6ca5e0..0b98a7f87cb8 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSource.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSource.java @@ -19,9 +19,10 @@ import com.facebook.presto.spi.Page; import com.facebook.presto.spi.PrestoException; 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.LazyBlock; import com.facebook.presto.spi.block.LazyBlockLoader; -import com.facebook.presto.spi.block.RunLengthEncodedBlock; import com.facebook.presto.spi.predicate.TupleDomain; import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.TypeManager; @@ -32,17 +33,13 @@ import parquet.schema.MessageType; import java.io.IOException; -import java.util.ArrayList; +import java.util.Arrays; import java.util.List; -import java.util.Optional; import java.util.Properties; import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveErrorCode.HIVE_CURSOR_ERROR; -import static com.facebook.presto.hive.parquet.ParquetTypeUtils.getDescriptor; -import static com.facebook.presto.hive.parquet.ParquetTypeUtils.getFieldIndex; import static com.facebook.presto.hive.parquet.ParquetTypeUtils.getParquetType; -import static com.facebook.presto.spi.type.StandardTypes.ROW; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import static java.util.Objects.requireNonNull; @@ -113,7 +110,11 @@ public ParquetPageSource( hiveColumnIndexes[columnIndex] = column.getHiveColumnIndex(); if (getParquetType(column, fileSchema, useParquetColumnNames) == null) { - constantBlocks[columnIndex] = RunLengthEncodedBlock.create(type, null, MAX_VECTOR_LENGTH); + BlockBuilder blockBuilder = type.createBlockBuilder(new BlockBuilderStatus(), MAX_VECTOR_LENGTH); + for (int i = 0; i < MAX_VECTOR_LENGTH; i++) { + blockBuilder.appendNull(); + } + constantBlocks[columnIndex] = blockBuilder.build(); } } types = typesBuilder.build(); @@ -168,38 +169,33 @@ public Page getNextPage() Block[] blocks = new Block[hiveColumnIndexes.length]; for (int fieldId = 0; fieldId < blocks.length; fieldId++) { + Type type = types.get(fieldId); if (constantBlocks[fieldId] != null) { blocks[fieldId] = constantBlocks[fieldId].getRegion(0, batchSize); } else { - Type type = types.get(fieldId); int fieldIndex; if (useParquetColumnNames) { - fieldIndex = getFieldIndex(fileSchema, columnNames.get(fieldId)); + fieldIndex = fileSchema.getFieldIndex(columnNames.get(fieldId)); } else { fieldIndex = hiveColumnIndexes[fieldId]; } - if (fieldIndex == -1) { - blocks[fieldId] = RunLengthEncodedBlock.create(type, null, batchSize); - continue; - } - - List path = new ArrayList<>(); - path.add(fileSchema.getFields().get(fieldIndex).getName()); - if (ROW.equals(type.getTypeSignature().getBase())) { - blocks[fieldId] = parquetReader.readStruct(type, path); - } - else { - Optional descriptor = getDescriptor(fileSchema, requestedSchema, path); - if (descriptor.isPresent()) { - blocks[fieldId] = new LazyBlock(batchSize, new ParquetBlockLoader(descriptor.get(), type)); - } - else { - blocks[fieldId] = RunLengthEncodedBlock.create(type, null, batchSize); + // Since we only support primitives in the new reader we just create the path + // from the field name and lookup the column descriptor with that path. + // With complex type support this lookup logic has to be rewritten. + parquet.schema.Type field = fileSchema.getFields().get(fieldIndex); + String[] path = new String[] {field.getName()}; + ColumnDescriptor columnDescriptor = null; + for (ColumnDescriptor column : fileSchema.getColumns()) { + if (Arrays.equals(column.getPath(), path)) { + columnDescriptor = column; + break; } } + checkState(columnDescriptor != null, "columnDescriptor is null"); + blocks[fieldId] = new LazyBlock(batchSize, new ParquetBlockLoader(columnDescriptor, type)); } } return new Page(batchSize, blocks); @@ -208,7 +204,7 @@ public Page getNextPage() closeWithSuppression(e); throw e; } - catch (IOException | RuntimeException e) { + catch (RuntimeException e) { closeWithSuppression(e); throw new PrestoException(HIVE_CURSOR_ERROR, e); } @@ -268,7 +264,7 @@ public final void load(LazyBlock lazyBlock) checkState(batchId == expectedBatchId); try { - Block block = parquetReader.readPrimitive(columnDescriptor, type); + Block block = parquetReader.readBlock(columnDescriptor, type); lazyBlock.setBlock(block); } catch (IOException 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 b13d44ce13d6..8edcc3790cf9 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 @@ -26,7 +26,6 @@ import com.facebook.presto.spi.predicate.TupleDomain; import com.facebook.presto.spi.type.TypeManager; import com.facebook.presto.spi.type.TypeSignature; -import com.facebook.presto.spi.type.TypeSignatureParameter; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableSet; import org.apache.hadoop.conf.Configuration; @@ -58,7 +57,6 @@ import static com.facebook.presto.hive.parquet.ParquetTypeUtils.getParquetType; import static com.facebook.presto.hive.parquet.predicate.ParquetPredicateUtils.buildParquetPredicate; import static com.facebook.presto.hive.parquet.predicate.ParquetPredicateUtils.predicateMatches; -import static com.facebook.presto.spi.type.StandardTypes.ARRAY; import static com.facebook.presto.spi.type.StandardTypes.BIGINT; import static com.facebook.presto.spi.type.StandardTypes.BOOLEAN; import static com.facebook.presto.spi.type.StandardTypes.CHAR; @@ -66,9 +64,7 @@ import static com.facebook.presto.spi.type.StandardTypes.DECIMAL; import static com.facebook.presto.spi.type.StandardTypes.DOUBLE; import static com.facebook.presto.spi.type.StandardTypes.INTEGER; -import static com.facebook.presto.spi.type.StandardTypes.MAP; import static com.facebook.presto.spi.type.StandardTypes.REAL; -import static com.facebook.presto.spi.type.StandardTypes.ROW; import static com.facebook.presto.spi.type.StandardTypes.SMALLINT; import static com.facebook.presto.spi.type.StandardTypes.TIMESTAMP; import static com.facebook.presto.spi.type.StandardTypes.TINYINT; @@ -86,7 +82,7 @@ public class ParquetPageSourceFactory .add("parquet.hive.serde.ParquetHiveSerDe") .build(); @VisibleForTesting - public static final Set SUPPORTED_COLUMN_TYPES = ImmutableSet.of(INTEGER, BIGINT, BOOLEAN, DOUBLE, REAL, TIMESTAMP, VARCHAR, CHAR, VARBINARY, DATE, DECIMAL, ROW); + public static final Set SUPPORTED_COLUMN_TYPES = ImmutableSet.of(INTEGER, BIGINT, BOOLEAN, DOUBLE, REAL, TIMESTAMP, VARCHAR, CHAR, VARBINARY, DATE, DECIMAL); private static final Set SUPPORTED_PARTITION_TYPES = ImmutableSet.of(TINYINT, SMALLINT, INTEGER, BIGINT, BOOLEAN, DOUBLE, REAL, TIMESTAMP, VARCHAR, CHAR, DATE, DECIMAL); private final TypeManager typeManager; @@ -192,11 +188,9 @@ public static ParquetPageSource createParquetPageSource( } ParquetReader parquetReader = new ParquetReader( - fileSchema, requestedSchema, blocks, - dataSource, - typeManager); + dataSource); return new ParquetPageSource( parquetReader, @@ -235,7 +229,8 @@ private static boolean columnTypeSupported(List columns) boolean regularColumnsSupported = columns.stream() .filter(column -> column.getColumnType() == REGULAR) .map(HiveColumnHandle::getTypeSignature) - .allMatch(ParquetPageSourceFactory::isTypeSupported); + .map(TypeSignature::getBase) + .allMatch(SUPPORTED_COLUMN_TYPES::contains); boolean partitionColumnsSupported = columns.stream() .filter(HiveColumnHandle::isPartitionKey) @@ -245,19 +240,4 @@ private static boolean columnTypeSupported(List columns) return regularColumnsSupported && partitionColumnsSupported; } - - private static boolean isTypeSupported(TypeSignature typeSignature) - { - if (MAP.equals(typeSignature.getBase()) || ARRAY.equals(typeSignature.getBase())) { - return false; - } - else if (ROW.equals(typeSignature.getBase())) { - return typeSignature.getParameters().stream() - .map(TypeSignatureParameter::getTypeSignatureOrNamedTypeSignature) - .filter(Optional::isPresent) - .map(Optional::get) - .allMatch(ParquetPageSourceFactory::isTypeSupported); - } - return true; - } } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java index 9daf58dd0957..a087fbcf1af2 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java @@ -14,74 +14,17 @@ package com.facebook.presto.hive.parquet; import com.facebook.presto.hive.HiveColumnHandle; -import parquet.column.ColumnDescriptor; import parquet.column.Encoding; -import parquet.io.ColumnIO; -import parquet.io.ColumnIOFactory; -import parquet.io.InvalidRecordException; import parquet.io.ParquetDecodingException; -import parquet.io.PrimitiveColumnIO; import parquet.schema.MessageType; import parquet.schema.Type; -import java.util.List; -import java.util.Optional; - -import static com.google.common.base.Preconditions.checkArgument; -import static java.util.Optional.empty; - public final class ParquetTypeUtils { private ParquetTypeUtils() { } - public static List getColumns(MessageType fileSchema, MessageType requestedSchema) - { - return (new ColumnIOFactory()).getColumnIO(requestedSchema, fileSchema, true).getLeaves(); - } - - public static Optional getDescriptor(MessageType fileSchema, MessageType requestedSchema, List path) - { - checkArgument(path.size() >= 1, "Parquet nested path should have at least one component"); - int level = path.size(); - for (PrimitiveColumnIO columnIO : getColumns(fileSchema, requestedSchema)) { - ColumnIO[] fields = columnIO.getPath(); - if (fields.length <= level) { - continue; - } - if (fields[level].getName().equalsIgnoreCase(path.get(level - 1))) { - boolean match = true; - for (int i = 0; i < level - 1; i++) { - if (!fields[i + 1].getName().equalsIgnoreCase(path.get(i))) { - match = false; - } - } - - if (match) { - ColumnDescriptor descriptor = columnIO.getColumnDescriptor(); - return Optional.of(new RichColumnDescriptor(descriptor.getPath(), columnIO.getType().asPrimitiveType(), descriptor.getMaxRepetitionLevel(), descriptor.getMaxDefinitionLevel())); - } - } - } - return empty(); - } - - public static int getFieldIndex(MessageType fileSchema, String name) - { - try { - return fileSchema.getFieldIndex(name); - } - catch (InvalidRecordException e) { - for (Type type : fileSchema.getFields()) { - if (type.getName().equalsIgnoreCase(name)) { - return fileSchema.getFieldIndex(type.getName()); - } - } - return -1; - } - } - public static parquet.schema.Type getParquetType(HiveColumnHandle column, MessageType messageType, boolean useParquetColumnNames) { if (useParquetColumnNames) { diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetColumnReader.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetColumnReader.java index 788b5f5dae1c..b55a9c94b36f 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetColumnReader.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/reader/ParquetColumnReader.java @@ -59,6 +59,7 @@ public abstract class ParquetColumnReader private int repetitionLevel; private int definitionLevel; private int currentValueCount; + private int pageValueCount; private ParquetDataPage page; private int remainingValueCountInPage; private int readOffset; @@ -156,7 +157,7 @@ public long getTotalValueCount() return totalValueCount; } - public Block readPrimitive(Type type) + public Block readBlock(Type type) throws IOException { checkArgument(currentValueCount <= totalValueCount, "Already read all values in column chunk"); @@ -199,6 +200,7 @@ private void readNextPage() { page = pageReader.readPage(); validateParquet(page != null, "Not enough values to read in column chunk"); + pageValueCount = page.getValueCount(); remainingValueCountInPage = page.getValueCount(); if (page instanceof ParquetDataPageV1) { @@ -227,11 +229,11 @@ private ValuesReader readPageV1(ParquetDataPageV1 page) definitionReader = new ParquetLevelValuesReader(dlReader); try { byte[] bytes = page.getSlice().getBytes(); - rlReader.initFromPage(page.getValueCount(), bytes, 0); + rlReader.initFromPage(pageValueCount, bytes, 0); int offset = rlReader.getNextOffset(); - dlReader.initFromPage(page.getValueCount(), bytes, offset); + dlReader.initFromPage(pageValueCount, bytes, offset); offset = dlReader.getNextOffset(); - return initDataReader(page.getValueEncoding(), bytes, offset, page.getValueCount()); + return initDataReader(page.getValueEncoding(), bytes, offset); } catch (IOException e) { throw new ParquetDecodingException("Error reading parquet page " + page + " in column " + columnDescriptor, e); @@ -242,7 +244,7 @@ private ValuesReader readPageV2(ParquetDataPageV2 page) { repetitionReader = buildLevelRLEReader(columnDescriptor.getMaxRepetitionLevel(), page.getRepetitionLevels()); definitionReader = buildLevelRLEReader(columnDescriptor.getMaxDefinitionLevel(), page.getDefinitionLevels()); - return initDataReader(page.getDataEncoding(), page.getSlice().getBytes(), 0, page.getValueCount()); + return initDataReader(page.getDataEncoding(), page.getSlice().getBytes(), 0); } private ParquetLevelReader buildLevelRLEReader(int maxLevel, Slice slice) @@ -253,7 +255,7 @@ private ParquetLevelReader buildLevelRLEReader(int maxLevel, Slice slice) return new ParquetLevelRLEReader(new RunLengthBitPackingHybridDecoder(BytesUtils.getWidthFromMaxInt(maxLevel), new ByteArrayInputStream(slice.getBytes()))); } - private ValuesReader initDataReader(ParquetEncoding dataEncoding, byte[] bytes, int offset, int valueCount) + private ValuesReader initDataReader(ParquetEncoding dataEncoding, byte[] bytes, int offset) { ValuesReader valuesReader; if (dataEncoding.usesDictionary()) { @@ -267,7 +269,7 @@ private ValuesReader initDataReader(ParquetEncoding dataEncoding, byte[] bytes, } try { - valuesReader.initFromPage(valueCount, bytes, offset); + valuesReader.initFromPage(pageValueCount, bytes, offset); return valuesReader; } catch (IOException e) { 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 79550ac6d537..d7ced9241eb7 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 @@ -16,32 +16,23 @@ import com.facebook.presto.hive.parquet.ParquetCorruptionException; import com.facebook.presto.hive.parquet.ParquetDataSource; import com.facebook.presto.hive.parquet.RichColumnDescriptor; -import com.facebook.presto.spi.block.ArrayBlock; 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.NamedTypeSignature; import com.facebook.presto.spi.type.Type; -import com.facebook.presto.spi.type.TypeManager; -import com.facebook.presto.spi.type.TypeSignatureParameter; import parquet.column.ColumnDescriptor; import parquet.hadoop.metadata.BlockMetaData; import parquet.hadoop.metadata.ColumnChunkMetaData; import parquet.hadoop.metadata.ColumnPath; -import parquet.io.PrimitiveColumnIO; import parquet.schema.MessageType; +import parquet.schema.PrimitiveType; import java.io.Closeable; import java.io.IOException; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; -import static com.facebook.presto.hive.parquet.ParquetTypeUtils.getColumns; -import static com.facebook.presto.hive.parquet.ParquetTypeUtils.getDescriptor; import static com.facebook.presto.hive.parquet.ParquetValidationUtils.validateParquet; -import static com.facebook.presto.spi.type.StandardTypes.ROW; import static com.google.common.primitives.Ints.checkedCast; import static java.lang.Math.min; @@ -50,31 +41,25 @@ public class ParquetReader { private static final int MAX_VECTOR_LENGTH = 1024; - private final MessageType fileSchema; private final MessageType requestedSchema; private final List blocks; private final ParquetDataSource dataSource; - private final TypeManager typeManager; private int currentBlock; private BlockMetaData currentBlockMetadata; private long currentPosition; private long currentGroupRowCount; private long nextRowInGroup; - private int batchSize; private final Map columnReadersMap = new HashMap<>(); - public ParquetReader(MessageType fileSchema, + public ParquetReader( MessageType requestedSchema, List blocks, - ParquetDataSource dataSource, - TypeManager typeManager) + ParquetDataSource dataSource) { - this.fileSchema = fileSchema; this.requestedSchema = requestedSchema; this.blocks = blocks; this.dataSource = dataSource; - this.typeManager = typeManager; initializeColumnReaders(); } @@ -96,13 +81,11 @@ public int nextBatch() return -1; } - batchSize = checkedCast(min(MAX_VECTOR_LENGTH, currentGroupRowCount - nextRowInGroup)); + int batchSize = checkedCast(min(MAX_VECTOR_LENGTH, currentGroupRowCount - nextRowInGroup)); nextRowInGroup += batchSize; currentPosition += batchSize; - for (PrimitiveColumnIO columnIO : getColumns(fileSchema, requestedSchema)) { - ColumnDescriptor descriptor = columnIO.getColumnDescriptor(); - RichColumnDescriptor column = new RichColumnDescriptor(descriptor.getPath(), columnIO.getType().asPrimitiveType(), descriptor.getMaxRepetitionLevel(), descriptor.getMaxDefinitionLevel()); + for (ColumnDescriptor column : getColumns(requestedSchema)) { ParquetColumnReader columnReader = columnReadersMap.get(column); columnReader.prepareNextRead(batchSize); } @@ -124,41 +107,7 @@ private boolean advanceToNextRowGroup() return true; } - public Block readStruct(Type type, List path) - throws IOException - { - List parameters = type.getTypeSignature().getParameters(); - Block[] blocks = new Block[parameters.size()]; - for (int i = 0; i < parameters.size(); i++) { - NamedTypeSignature namedTypeSignature = parameters.get(i).getNamedTypeSignature(); - Type fieldType = typeManager.getType(namedTypeSignature.getTypeSignature()); - String name = namedTypeSignature.getName(); - path.add(name); - Optional columnDescriptor = getDescriptor(fileSchema, requestedSchema, path); - if (!columnDescriptor.isPresent()) { - path.remove(name); - blocks[i] = RunLengthEncodedBlock.create(type, null, batchSize); - continue; - } - - if (ROW.equals(fieldType.getTypeSignature().getBase())) { - blocks[i] = readStruct(fieldType, path); - } - else { - blocks[i] = readPrimitive(columnDescriptor.get(), fieldType); - } - path.remove(name); - } - - InterleavedBlock interleavedBlock = new InterleavedBlock(blocks); - int[] offsets = new int[batchSize + 1]; - for (int i = 1; i < offsets.length; i++) { - offsets[i] = i * parameters.size(); - } - return new ArrayBlock(batchSize, new boolean[batchSize], offsets, interleavedBlock); - } - - public Block readPrimitive(ColumnDescriptor columnDescriptor, Type type) + public Block readBlock(ColumnDescriptor columnDescriptor, Type type) throws IOException { ParquetColumnReader columnReader = columnReadersMap.get(columnDescriptor); @@ -173,7 +122,7 @@ public Block readPrimitive(ColumnDescriptor columnDescriptor, Type type) ParquetColumnChunk columnChunk = new ParquetColumnChunk(descriptor, buffer, 0); columnReader.setPageReader(columnChunk.readAllPages()); } - return columnReader.readPrimitive(type); + return columnReader.readBlock(type); } private ColumnChunkMetaData getColumnChunkMetaData(ColumnDescriptor columnDescriptor) @@ -189,10 +138,23 @@ private ColumnChunkMetaData getColumnChunkMetaData(ColumnDescriptor columnDescri private void initializeColumnReaders() { - for (PrimitiveColumnIO columnIO : getColumns(fileSchema, requestedSchema)) { - ColumnDescriptor descriptor = columnIO.getColumnDescriptor(); - RichColumnDescriptor column = new RichColumnDescriptor(descriptor.getPath(), columnIO.getType().asPrimitiveType(), descriptor.getMaxRepetitionLevel(), descriptor.getMaxDefinitionLevel()); + for (RichColumnDescriptor column : getColumns(requestedSchema)) { columnReadersMap.put(column, ParquetColumnReader.createReader(column)); } } + + private static List getColumns(MessageType schema) + { + List paths = schema.getPaths(); + List columns = new ArrayList<>(paths.size()); + for (String[] path : paths) { + PrimitiveType primitiveType = schema.getType(path).asPrimitiveType(); + columns.add(new RichColumnDescriptor( + path, + primitiveType, + schema.getMaxRepetitionLevel(path), + schema.getMaxDefinitionLevel(path))); + } + return columns; + } } diff --git a/presto-hive/src/main/java/org/apache/hadoop/security/UserGroupInformationShim.java b/presto-hive/src/main/java/org/apache/hadoop/security/UserGroupInformationShim.java new file mode 100644 index 000000000000..0e828c51fa9e --- /dev/null +++ b/presto-hive/src/main/java/org/apache/hadoop/security/UserGroupInformationShim.java @@ -0,0 +1,52 @@ +/* + * 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 org.apache.hadoop.security; + +import javax.security.auth.Subject; +import javax.security.auth.kerberos.KerberosPrincipal; + +import java.util.Set; + +import static org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod.KERBEROS; + +public final class UserGroupInformationShim +{ + private UserGroupInformationShim() {} + + public static Subject getSubject(UserGroupInformation userGroupInformation) + { + return userGroupInformation.getSubject(); + } + + public static UserGroupInformation createUserGroupInformationForSubject(Subject subject) + { + if (subject == null) { + throw new NullPointerException("subject is null"); + } + Set kerberosPrincipals = subject.getPrincipals(KerberosPrincipal.class); + if (kerberosPrincipals.isEmpty()) { + throw new IllegalArgumentException("subject must contain a KerberosPrincipal"); + } + if (kerberosPrincipals.size() != 1) { + throw new IllegalArgumentException("subject must contain only a single KerberosPrincipal"); + } + + KerberosPrincipal principal = kerberosPrincipals.iterator().next(); + User user = new User(principal.getName(), KERBEROS, null); + subject.getPrincipals().add(user); + UserGroupInformation userGroupInformation = new UserGroupInformation(subject); + userGroupInformation.setAuthenticationMethod(KERBEROS); + return userGroupInformation; + } +} 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 42542932dc95..4b8f003d99ef 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 @@ -147,7 +147,7 @@ import static com.facebook.presto.hive.HiveType.HIVE_LONG; import static com.facebook.presto.hive.HiveType.HIVE_STRING; import static com.facebook.presto.hive.HiveType.toHiveType; -import static com.facebook.presto.hive.HiveUtil.columnExtraInfo; +import static com.facebook.presto.hive.HiveUtil.annotateColumnComment; import static com.facebook.presto.hive.HiveWriteUtils.createDirectory; import static com.facebook.presto.hive.util.Types.checkType; import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; @@ -414,7 +414,7 @@ protected void setupHive(String connectorId, String databaseName, String timeZon invalidTableHandle = new HiveTableHandle(invalidClientId, database, INVALID_TABLE); invalidTableLayoutHandle = new HiveTableLayoutHandle(invalidClientId, ImmutableList.of(), - ImmutableList.of(new HivePartition(invalidTable, TupleDomain.all(), "unknown", ImmutableMap.of(), ImmutableList.of())), + ImmutableList.of(new HivePartition(invalidTable, TupleDomain.all(), "unknown", ImmutableMap.of(), Optional.empty())), TupleDomain.all(), Optional.empty()); emptyTableLayoutHandle = new HiveTableLayoutHandle(invalidClientId, ImmutableList.of(), ImmutableList.of(), TupleDomain.none(), Optional.empty()); @@ -428,41 +428,41 @@ protected void setupHive(String connectorId, String databaseName, String timeZon List partitionColumns = ImmutableList.of(dsColumn, fileFormatColumn, dummyColumn); List partitions = ImmutableList.builder() .add(new HivePartition(tablePartitionFormat, - TupleDomain.all(), + TupleDomain.all(), "ds=2012-12-29/file_format=textfile/dummy=1", ImmutableMap.builder() .put(dsColumn, NullableValue.of(createUnboundedVarcharType(), utf8Slice("2012-12-29"))) .put(fileFormatColumn, NullableValue.of(createUnboundedVarcharType(), utf8Slice("textfile"))) .put(dummyColumn, NullableValue.of(INTEGER, 1L)) .build(), - ImmutableList.of())) + Optional.empty())) .add(new HivePartition(tablePartitionFormat, - TupleDomain.all(), + TupleDomain.all(), "ds=2012-12-29/file_format=sequencefile/dummy=2", ImmutableMap.builder() .put(dsColumn, NullableValue.of(createUnboundedVarcharType(), utf8Slice("2012-12-29"))) .put(fileFormatColumn, NullableValue.of(createUnboundedVarcharType(), utf8Slice("sequencefile"))) .put(dummyColumn, NullableValue.of(INTEGER, 2L)) .build(), - ImmutableList.of())) + Optional.empty())) .add(new HivePartition(tablePartitionFormat, - TupleDomain.all(), + TupleDomain.all(), "ds=2012-12-29/file_format=rctext/dummy=3", ImmutableMap.builder() .put(dsColumn, NullableValue.of(createUnboundedVarcharType(), utf8Slice("2012-12-29"))) .put(fileFormatColumn, NullableValue.of(createUnboundedVarcharType(), utf8Slice("rctext"))) .put(dummyColumn, NullableValue.of(INTEGER, 3L)) .build(), - ImmutableList.of())) + Optional.empty())) .add(new HivePartition(tablePartitionFormat, - TupleDomain.all(), + TupleDomain.all(), "ds=2012-12-29/file_format=rcbinary/dummy=4", ImmutableMap.builder() .put(dsColumn, NullableValue.of(createUnboundedVarcharType(), utf8Slice("2012-12-29"))) .put(fileFormatColumn, NullableValue.of(createUnboundedVarcharType(), utf8Slice("rcbinary"))) .put(dummyColumn, NullableValue.of(INTEGER, 4L)) .build(), - ImmutableList.of())) + Optional.empty())) .build(); partitionCount = partitions.size(); tupleDomain = TupleDomain.fromFixedValues(ImmutableMap.of(dsColumn, NullableValue.of(createUnboundedVarcharType(), utf8Slice("2012-12-29")))); @@ -494,7 +494,7 @@ protected void setupHive(String connectorId, String databaseName, String timeZon dummyColumn, Domain.create(ValueSet.ofRanges(Range.equal(INTEGER, 4L)), false))) ))), ImmutableList.of()); - List unpartitionedPartitions = ImmutableList.of(new HivePartition(tableUnpartitioned, TupleDomain.all(), ImmutableList.of())); + List unpartitionedPartitions = ImmutableList.of(new HivePartition(tableUnpartitioned, TupleDomain.all())); unpartitionedTableLayout = new ConnectorTableLayout(new HiveTableLayoutHandle(clientId, ImmutableList.of(), unpartitionedPartitions, TupleDomain.all(), Optional.empty())); timeZone = DateTimeZone.forTimeZone(TimeZone.getTimeZone(timeZoneId)); } @@ -516,7 +516,7 @@ protected final void setup(String host, int port, String databaseName, String ti } HiveCluster hiveCluster = new TestingHiveCluster(hiveClientConfig, host, port); - metastoreClient = new CachingHiveMetastore(new BridgingHiveMetastore(new ThriftHiveMetastore(hiveCluster)), executor, Duration.valueOf("1m"), Duration.valueOf("15s"), 10000); + metastoreClient = new CachingHiveMetastore(new BridgingHiveMetastore(new ThriftHiveMetastore(hiveCluster)), executor, Duration.valueOf("1m"), Duration.valueOf("15s")); HiveConnectorId connectorId = new HiveConnectorId(connectorName); HdfsConfiguration hdfsConfiguration = new HiveHdfsConfiguration(new HdfsConfigurationUpdater(hiveClientConfig)); @@ -535,8 +535,8 @@ protected final void setup(String host, int port, String databaseName, String ti true, false, true, + true, HiveStorageFormat.RCBINARY, - 1000, typeManager, locationService, new TableParameterCodec(), @@ -799,7 +799,7 @@ public void testMismatchSchemaTable() } } - protected void doTestMismatchSchemaTable( + private void doTestMismatchSchemaTable( SchemaTableName schemaTableName, HiveStorageFormat storageFormat, List tableBefore, @@ -944,7 +944,8 @@ protected void assertExpectedPartitions(List actualPartitions, Iterable ex assertEquals(actualPartition.getPartitionId(), expectedPartition.getPartitionId()); assertEquals(actualPartition.getKeys(), expectedPartition.getKeys()); assertEquals(actualPartition.getTableName(), expectedPartition.getTableName()); - assertEquals(actualPartition.getBuckets(), expectedPartition.getBuckets()); + assertEquals(actualPartition.getBucket(), expectedPartition.getBucket()); + assertEquals(actualPartition.getTupleDomain(), expectedPartition.getTupleDomain()); } } @@ -1536,7 +1537,7 @@ public void testTypesRcTextRecordCursor() new HiveClientConfig().setTimeZone(timeZone.getID()), hdfsEnvironment, ImmutableSet.of(new ColumnarTextHiveRecordCursorProvider(hdfsEnvironment)), - ImmutableSet.of(), + ImmutableSet.of(), TYPE_MANAGER); ConnectorPageSource pageSource = pageSourceProvider.createPageSource(transaction.getTransactionHandle(), session, hiveSplit, columnHandles); @@ -1572,7 +1573,7 @@ public void testTypesRcBinaryRecordCursor() new HiveClientConfig().setTimeZone(timeZone.getID()), hdfsEnvironment, ImmutableSet.of(new ColumnarBinaryHiveRecordCursorProvider(hdfsEnvironment)), - ImmutableSet.of(), + ImmutableSet.of(), TYPE_MANAGER); ConnectorPageSource pageSource = pageSourceProvider.createPageSource(transaction.getTransactionHandle(), session, hiveSplit, columnHandles); @@ -1991,8 +1992,7 @@ protected void doCreateEmptyTable(SchemaTableName tableName, HiveStorageFormat s .map(column -> new ColumnMetadata( column.getName(), column.getType(), - column.getComment(), - columnExtraInfo(partitionedBy.contains(column.getName())), + annotateColumnComment(Optional.ofNullable(column.getComment()), partitionedBy.contains(column.getName())), false)) .collect(toList()); assertEquals(filterNonHiddenColumnMetadata(tableMetadata.getColumns()), expectedColumns); @@ -2035,7 +2035,7 @@ private void doInsert(HiveStorageFormat storageFormat, SchemaTableName tableName // verify the data resultBuilder.rows(CREATE_TABLE_DATA.getMaterializedRows()); - MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.empty(), Optional.empty()); + MaterializedResult result = readTable(transaction, tableHandle, columnHandles, session, TupleDomain.all(), OptionalInt.empty(), Optional.empty()); assertEqualsIgnoreOrder(result.getMaterializedRows(), resultBuilder.build().getMaterializedRows()); } } @@ -2717,8 +2717,8 @@ else if (rowNumber % 39 == 1) { assertNull(row.getField(index)); } else { - List expected1 = ImmutableList.of("test abc", 0.1); - List expected2 = ImmutableList.of("test xyz", 0.2); + List expected1 = ImmutableList.of("test abc", 0.1); + List expected2 = ImmutableList.of("test xyz", 0.2); assertEquals(row.getField(index), ImmutableList.of(expected1, expected2)); } } @@ -2730,8 +2730,8 @@ else if (rowNumber % 39 == 1) { assertNull(row.getField(index)); } else { - List expected1 = ImmutableList.of("test abc", 0.1); - List expected2 = ImmutableList.of("test xyz", 0.2); + List expected1 = ImmutableList.of("test abc", 0.1); + List expected2 = ImmutableList.of("test xyz", 0.2); assertEquals(row.getField(index), ImmutableMap.of(1, ImmutableList.of(expected1, expected2))); } } @@ -2974,7 +2974,7 @@ private static void assertPrimitiveField(Map map, String assertTrue(map.containsKey(name)); ColumnMetadata column = map.get(name); assertEquals(column.getType(), type, name); - assertEquals(column.getExtraInfo(), columnExtraInfo(partitionKey)); + assertEquals(column.getComment(), annotateColumnComment(Optional.empty(), partitionKey)); } protected static ImmutableMap indexColumns(List columnHandles) 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 bb6697ccd907..0307a5604d05 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 @@ -407,7 +407,7 @@ public abstract class AbstractTestHiveFileFormats getStandardListObjectInspector( getStandardStructObjectInspector( ImmutableList.of("s_int"), - ImmutableList.of(javaIntObjectInspector) + ImmutableList.of(javaIntObjectInspector) ) ) ), @@ -431,9 +431,9 @@ public abstract class AbstractTestHiveFileFormats mapBlockOf(new ArrayType(createUnboundedVarcharType()), new MapType(BIGINT, BOOLEAN), arrayBlockOf(createUnboundedVarcharType(), "k", "ka"), mapBlockOf(BIGINT, BOOLEAN, 16L, false)))) .add(new TestColumn("t_struct_nested", getStandardStructObjectInspector(ImmutableList.of("struct_field"), ImmutableList.of(getStandardListObjectInspector(javaStringObjectInspector))), ImmutableList.of(ImmutableList.of("1", "2", "3")), rowBlockOf(ImmutableList.of(new ArrayType(createUnboundedVarcharType())), arrayBlockOf(createUnboundedVarcharType(), "1", "2", "3")))) - .add(new TestColumn("t_struct_null", getStandardStructObjectInspector(ImmutableList.of("struct_field_null", "struct_field_null2"), + .add(new TestColumn("t_struct_null", getStandardStructObjectInspector(ImmutableList.of("struct_field", "struct_field2"), ImmutableList.of(javaStringObjectInspector, javaStringObjectInspector)), Arrays.asList(null, null), rowBlockOf(ImmutableList.of(createUnboundedVarcharType(), createUnboundedVarcharType()), null, null))) - .add(new TestColumn("t_struct_non_nulls_after_nulls", getStandardStructObjectInspector(ImmutableList.of("struct_non_nulls_after_nulls1", "struct_non_nulls_after_nulls2"), + .add(new TestColumn("t_struct_non_nulls_after_nulls", getStandardStructObjectInspector(ImmutableList.of("struct_field1", "struct_field2"), ImmutableList.of(javaIntObjectInspector, javaStringObjectInspector)), Arrays.asList(null, "some string"), rowBlockOf(ImmutableList.of(INTEGER, createUnboundedVarcharType()), null, "some string"))) .add(new TestColumn("t_nested_struct_non_nulls_after_nulls", getStandardStructObjectInspector( diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/HiveBenchmarkQueryRunner.java b/presto-hive/src/test/java/com/facebook/presto/hive/HiveBenchmarkQueryRunner.java index 8dbf4a5562dc..e4bb581736ce 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/HiveBenchmarkQueryRunner.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/HiveBenchmarkQueryRunner.java @@ -60,7 +60,7 @@ public static LocalQueryRunner createLocalQueryRunner(File tempDir) LocalQueryRunner localQueryRunner = new LocalQueryRunner(session); // add tpch - localQueryRunner.createCatalog("tpch", new TpchConnectorFactory(1), ImmutableMap.of()); + localQueryRunner.createCatalog("tpch", new TpchConnectorFactory(1), ImmutableMap.of()); // add hive File hiveDir = new File(tempDir, "hive_data"); 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 46047e64fac1..7d2a70614742 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 @@ -38,14 +38,11 @@ public void testDefaults() ConfigAssertions.assertRecordedDefaults(ConfigAssertions.recordDefaults(HiveClientConfig.class) .setTimeZone(TimeZone.getDefault().getID()) .setMaxSplitSize(new DataSize(64, Unit.MEGABYTE)) - .setMaxPartitionsPerScan(100_000) .setMaxOutstandingSplits(1_000) .setMaxSplitIteratorThreads(1_000) .setAllowCorruptWritesForTesting(false) .setMetastoreCacheTtl(new Duration(1, TimeUnit.HOURS)) .setMetastoreRefreshInterval(new Duration(1, TimeUnit.SECONDS)) - .setMetastoreCacheMaximumSize(10000) - .setPerTransactionMetastoreCacheMaximumSize(1000) .setMaxMetastoreRefreshThreads(100) .setMetastoreSocksProxy(null) .setMetastoreTimeout(new Duration(10, TimeUnit.SECONDS)) @@ -111,8 +108,7 @@ public void testDefaults() .setHdfsPrestoKeytab(null) .setSkipDeletionForAlter(false) .setBucketExecutionEnabled(true) - .setBucketWritingEnabled(true) - .setFileSystemMaxCacheSize(1000)); + .setBucketWritingEnabled(true)); } @Test @@ -121,14 +117,11 @@ public void testExplicitPropertyMappings() Map properties = new ImmutableMap.Builder() .put("hive.time-zone", nonDefaultTimeZone().getID()) .put("hive.max-split-size", "256MB") - .put("hive.max-partitions-per-scan", "123") .put("hive.max-outstanding-splits", "10") .put("hive.max-split-iterator-threads", "10") .put("hive.allow-corrupt-writes-for-testing", "true") .put("hive.metastore-cache-ttl", "2h") .put("hive.metastore-refresh-interval", "30m") - .put("hive.metastore-cache-maximum-size", "5000") - .put("hive.per-transaction-metastore-cache-maximum-size", "500") .put("hive.metastore-refresh-max-threads", "2500") .put("hive.metastore.thrift.client.socks-proxy", "localhost:1080") .put("hive.metastore-timeout", "20s") @@ -195,20 +188,16 @@ public void testExplicitPropertyMappings() .put("hive.skip-deletion-for-alter", "true") .put("hive.bucket-execution", "false") .put("hive.bucket-writing", "false") - .put("hive.fs.cache.max-size", "1010") .build(); HiveClientConfig expected = new HiveClientConfig() .setTimeZone(nonDefaultTimeZone().toTimeZone().getID()) .setMaxSplitSize(new DataSize(256, Unit.MEGABYTE)) - .setMaxPartitionsPerScan(123) .setMaxOutstandingSplits(10) .setMaxSplitIteratorThreads(10) .setAllowCorruptWritesForTesting(true) .setMetastoreCacheTtl(new Duration(2, TimeUnit.HOURS)) .setMetastoreRefreshInterval(new Duration(30, TimeUnit.MINUTES)) - .setMetastoreCacheMaximumSize(5000) - .setPerTransactionMetastoreCacheMaximumSize(500) .setMaxMetastoreRefreshThreads(2500) .setMetastoreSocksProxy(HostAndPort.fromParts("localhost", 1080)) .setMetastoreTimeout(new Duration(20, TimeUnit.SECONDS)) @@ -274,8 +263,7 @@ public void testExplicitPropertyMappings() .setHdfsPrestoKeytab("/tmp/presto.keytab") .setSkipDeletionForAlter(true) .setBucketExecutionEnabled(false) - .setBucketWritingEnabled(false) - .setFileSystemMaxCacheSize(1010); + .setBucketWritingEnabled(false); ConfigAssertions.assertFullMapping(properties, expected); } diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveDistributedQueries.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveDistributedQueries.java index e2ba7017c6c8..b3cfa8d1ad25 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveDistributedQueries.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestHiveDistributedQueries.java @@ -13,15 +13,10 @@ */ package com.facebook.presto.hive; -import com.facebook.presto.testing.MaterializedResult; import com.facebook.presto.tests.AbstractTestDistributedQueries; -import org.testng.annotations.Test; import static com.facebook.presto.hive.HiveQueryRunner.createQueryRunner; -import static com.facebook.presto.spi.type.CharType.createCharType; -import static com.facebook.presto.testing.MaterializedResult.resultBuilder; import static io.airlift.tpch.TpchTable.getTables; -import static org.testng.Assert.assertEquals; public class TestHiveDistributedQueries extends AbstractTestDistributedQueries @@ -55,28 +50,4 @@ public void testRenameTable() { // Hive connector currently does not support table rename } - - @Test - public void testOrderByChar() - throws Exception - { - assertUpdate("CREATE TABLE char_order_by (c_char char(2))"); - assertUpdate("INSERT INTO char_order_by (c_char) VALUES" + - "(CAST('a' as CHAR(2)))," + - "(CAST('a\0' as CHAR(2)))," + - "(CAST('a ' as CHAR(2)))", 3); - - MaterializedResult actual = computeActual(getSession(), - "SELECT * FROM char_order_by ORDER BY c_char ASC"); - - assertUpdate("DROP TABLE char_order_by"); - - MaterializedResult expected = resultBuilder(getSession(), createCharType(2)) - .row("a\0") - .row("a ") - .row("a ") - .build(); - - assertEquals(actual, expected); - } } 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 eebdd3900c48..0ff2a2d0e7ce 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 @@ -59,7 +59,6 @@ import java.io.IOException; import java.util.List; import java.util.Optional; -import java.util.OptionalInt; import java.util.Properties; import java.util.TimeZone; import java.util.stream.Collectors; @@ -107,7 +106,6 @@ public class TestHiveFileFormats private static TestingConnectorSession parquetCursorPushdownSession = new TestingConnectorSession(new HiveSessionProperties(new HiveClientConfig().setParquetOptimizedReaderEnabled(false).setParquetPredicatePushdownEnabled(true)).getSessionProperties()); private static TestingConnectorSession parquetPageSourceSession = new TestingConnectorSession(new HiveSessionProperties(new HiveClientConfig().setParquetOptimizedReaderEnabled(true).setParquetPredicatePushdownEnabled(false)).getSessionProperties()); private static TestingConnectorSession parquetPageSourcePushdown = new TestingConnectorSession(new HiveSessionProperties(new HiveClientConfig().setParquetOptimizedReaderEnabled(true).setParquetPredicatePushdownEnabled(true)).getSessionProperties()); - private static final List SUPPORTED_PARQUET_STRUCT_COLUMNS = ImmutableList.of("t_struct_bigint", "t_struct_null", "t_struct_non_nulls_after_nulls", "t_nested_struct_non_nulls_after_nulls"); @DataProvider(name = "rowCount") public static Object[][] rowCountProvider() @@ -282,7 +280,7 @@ public void testParquetPageSource(int rowCount) { List testColumns = getTestColumnsSupportedByParquet(); testColumns = testColumns.stream() - .filter(column -> column.getObjectInspector().getCategory() == Category.PRIMITIVE || SUPPORTED_PARQUET_STRUCT_COLUMNS.contains(column.getName())) + .filter(column -> column.getObjectInspector().getCategory() == Category.PRIMITIVE) .collect(toList()); assertThatFileFormat(PARQUET) .withColumns(testColumns) @@ -572,7 +570,6 @@ private void testCursorProvider(HiveRecordCursorProvider cursorProvider, new Configuration(), SESSION, split.getPath(), - OptionalInt.empty(), split.getStart(), split.getLength(), splitProperties, @@ -617,7 +614,6 @@ private void testPageSourceFactory(HivePageSourceFactory sourceFactory, new Configuration(), session, split.getPath(), - OptionalInt.empty(), split.getStart(), split.getLength(), splitProperties, 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 6150c3a80e4e..7f6e7f940174 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 @@ -21,6 +21,7 @@ import com.facebook.presto.metadata.TableLayoutResult; import com.facebook.presto.metadata.TableMetadata; import com.facebook.presto.spi.ColumnMetadata; +import com.facebook.presto.spi.ConnectorTableLayoutHandle; import com.facebook.presto.spi.Constraint; import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.TypeManager; @@ -47,9 +48,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.function.Function; -import static com.facebook.presto.hive.HiveColumnHandle.BUCKET_COLUMN_NAME; import static com.facebook.presto.hive.HiveColumnHandle.PATH_COLUMN_NAME; import static com.facebook.presto.hive.HiveQueryRunner.HIVE_CATALOG; import static com.facebook.presto.hive.HiveQueryRunner.TPCH_SCHEMA; @@ -59,7 +58,7 @@ import static com.facebook.presto.hive.HiveTableProperties.BUCKET_COUNT_PROPERTY; import static com.facebook.presto.hive.HiveTableProperties.PARTITIONED_BY_PROPERTY; import static com.facebook.presto.hive.HiveTableProperties.STORAGE_FORMAT_PROPERTY; -import static com.facebook.presto.hive.HiveUtil.columnExtraInfo; +import static com.facebook.presto.hive.HiveUtil.annotateColumnComment; import static com.facebook.presto.spi.type.BigintType.BIGINT; import static com.facebook.presto.spi.type.CharType.createCharType; import static com.facebook.presto.spi.type.DecimalType.createDecimalType; @@ -113,9 +112,9 @@ protected TestHiveIntegrationSmokeTest(QueryRunner queryRunner, Session bucketed this.typeManager = new TypeRegistry(); } - protected List getPartitions(HiveTableLayoutHandle tableLayoutHandle) + protected List getPartitions(ConnectorTableLayoutHandle tableLayoutHandle) { - return tableLayoutHandle.getPartitions().get(); + return ((HiveTableLayoutHandle) tableLayoutHandle).getPartitions().get(); } @Test @@ -263,7 +262,7 @@ public void createPartitionedTable(Session session, HiveStorageFormat storageFor assertEquals(tableMetadata.getMetadata().getProperties().get(PARTITIONED_BY_PROPERTY), partitionedBy); for (ColumnMetadata columnMetadata : tableMetadata.getColumns()) { boolean partitionKey = partitionedBy.contains(columnMetadata.getName()); - assertEquals(columnMetadata.getExtraInfo(), columnExtraInfo(partitionKey)); + assertEquals(columnMetadata.getComment(), annotateColumnComment(Optional.empty(), partitionKey)); } assertColumnType(tableMetadata, "_string", createUnboundedVarcharType()); @@ -356,6 +355,10 @@ protected void createTableLike(String likeSuffix, boolean hasPartition) // Verify the partition keys are correctly created List partitionedBy = ImmutableList.of("partition_bigint", "partition_decimal_long"); assertEquals(tableMetadata.getMetadata().getProperties().get(PARTITIONED_BY_PROPERTY), partitionedBy); + for (ColumnMetadata columnMetadata : tableMetadata.getColumns()) { + boolean partitionKey = partitionedBy.contains(columnMetadata.getName()); + assertEquals(columnMetadata.getComment(), annotateColumnComment(Optional.empty(), partitionKey)); + } // Verify the column types assertColumnType(tableMetadata, "string_col", createUnboundedVarcharType()); @@ -500,7 +503,13 @@ public void createPartitionedTableAs(Session session, HiveStorageFormat storageF TableMetadata tableMetadata = getTableMetadata(catalog, TPCH_SCHEMA, "test_create_partitioned_table_as"); assertEquals(tableMetadata.getMetadata().getProperties().get(STORAGE_FORMAT_PROPERTY), storageFormat); - assertEquals(tableMetadata.getMetadata().getProperties().get(PARTITIONED_BY_PROPERTY), ImmutableList.of("ship_priority", "order_status")); + + List partitionedBy = ImmutableList.of("ship_priority", "order_status"); + assertEquals(tableMetadata.getMetadata().getProperties().get(PARTITIONED_BY_PROPERTY), partitionedBy); + for (ColumnMetadata columnMetadata : tableMetadata.getColumns()) { + boolean partitionKey = partitionedBy.contains(columnMetadata.getName()); + assertEquals(columnMetadata.getComment(), annotateColumnComment(Optional.empty(), partitionKey)); + } List partitions = getPartitions("test_create_partitioned_table_as"); assertEquals(partitions.size(), 3); @@ -692,7 +701,13 @@ private void verifyPartitionedBucketedTable(HiveStorageFormat storageFormat, Str TableMetadata tableMetadata = getTableMetadata(catalog, TPCH_SCHEMA, tableName); assertEquals(tableMetadata.getMetadata().getProperties().get(STORAGE_FORMAT_PROPERTY), storageFormat); - assertEquals(tableMetadata.getMetadata().getProperties().get(PARTITIONED_BY_PROPERTY), ImmutableList.of("orderstatus")); + List partitionedBy = ImmutableList.of("orderstatus"); + assertEquals(tableMetadata.getMetadata().getProperties().get(PARTITIONED_BY_PROPERTY), partitionedBy); + for (ColumnMetadata columnMetadata : tableMetadata.getColumns()) { + boolean partitionKey = partitionedBy.contains(columnMetadata.getName()); + assertEquals(columnMetadata.getComment(), annotateColumnComment(Optional.empty(), partitionKey)); + } + assertEquals(tableMetadata.getMetadata().getProperties().get(BUCKETED_BY_PROPERTY), ImmutableList.of("custkey", "custkey2")); assertEquals(tableMetadata.getMetadata().getProperties().get(BUCKET_COUNT_PROPERTY), 11); @@ -822,7 +837,13 @@ private void verifyPartitionedBucketedTableAsFewRows(HiveStorageFormat storageFo TableMetadata tableMetadata = getTableMetadata(catalog, TPCH_SCHEMA, tableName); assertEquals(tableMetadata.getMetadata().getProperties().get(STORAGE_FORMAT_PROPERTY), storageFormat); - assertEquals(tableMetadata.getMetadata().getProperties().get(PARTITIONED_BY_PROPERTY), ImmutableList.of("partition_key")); + List partitionedBy = ImmutableList.of("partition_key"); + assertEquals(tableMetadata.getMetadata().getProperties().get(PARTITIONED_BY_PROPERTY), partitionedBy); + for (ColumnMetadata columnMetadata : tableMetadata.getColumns()) { + boolean partitionKey = partitionedBy.contains(columnMetadata.getName()); + assertEquals(columnMetadata.getComment(), annotateColumnComment(Optional.empty(), partitionKey)); + } + assertEquals(tableMetadata.getMetadata().getProperties().get(BUCKETED_BY_PROPERTY), ImmutableList.of("bucket_key")); assertEquals(tableMetadata.getMetadata().getProperties().get(BUCKET_COUNT_PROPERTY), 11); @@ -1278,7 +1299,7 @@ private TableMetadata getTableMetadata(String catalog, String schema, String tab }); } - private Object getHiveTableProperty(String tableName, Function propertyGetter) + private List getPartitions(String tableName) { Session session = getSession(); Metadata metadata = ((DistributedQueryRunner) queryRunner).getCoordinator().getMetadata(); @@ -1291,20 +1312,10 @@ private Object getHiveTableProperty(String tableName, Function layouts = metadata.getLayouts(transactionSession, tableHandle.get(), Constraint.alwaysTrue(), Optional.empty()); TableLayout layout = getOnlyElement(layouts).getLayout(); - return propertyGetter.apply((HiveTableLayoutHandle) layout.getHandle().getConnectorHandle()); + return getPartitions(layout.getHandle().getConnectorHandle()); }); } - private List getPartitions(String tableName) - { - return (List) getHiveTableProperty(tableName, (HiveTableLayoutHandle table) -> getPartitions(table)); - } - - private int getBucketCount(String tableName) - { - return (int) getHiveTableProperty(tableName, (HiveTableLayoutHandle table) -> table.getBucketHandle().get().getBucketCount()); - } - @Test public void testShowColumnsPartitionKey() { @@ -1315,14 +1326,14 @@ public void testShowColumnsPartitionKey() MaterializedResult actual = computeActual("SHOW COLUMNS FROM test_show_columns_partition_key"); MaterializedResult expected = resultBuilder(getSession(), canonicalizeType(createUnboundedVarcharType()), canonicalizeType(createUnboundedVarcharType()), canonicalizeType(createUnboundedVarcharType())) - .row("grape", canonicalizeTypeName("bigint"), "", "") - .row("orange", canonicalizeTypeName("bigint"), "", "") - .row("pear", canonicalizeTypeName("varchar(65535)"), "", "") - .row("mango", canonicalizeTypeName("integer"), "", "") - .row("lychee", canonicalizeTypeName("smallint"), "", "") - .row("kiwi", canonicalizeTypeName("tinyint"), "", "") - .row("apple", canonicalizeTypeName("varchar"), "partition key", "") - .row("pineapple", canonicalizeTypeName("varchar(65535)"), "partition key", "") + .row("grape", canonicalizeTypeName("bigint"), "") + .row("orange", canonicalizeTypeName("bigint"), "") + .row("pear", canonicalizeTypeName("varchar(65535)"), "") + .row("mango", canonicalizeTypeName("integer"), "") + .row("lychee", canonicalizeTypeName("smallint"), "") + .row("kiwi", canonicalizeTypeName("tinyint"), "") + .row("apple", canonicalizeTypeName("varchar"), "Partition Key") + .row("pineapple", canonicalizeTypeName("varchar(65535)"), "Partition Key") .build(); assertEquals(actual, expected); } @@ -1617,60 +1628,6 @@ private void doTestPathHiddenColumn(Session session, HiveStorageFormat storageFo assertFalse(queryRunner.tableExists(session, "test_path")); } - @Test - public void testBucketHiddenColumn() - throws Exception - { - @Language("SQL") String createTable = "CREATE TABLE test_bucket_hidden_column " + - "WITH (" + - "bucketed_by = ARRAY['col0']," + - "bucket_count = 2" + - ") AS " + - "SELECT * FROM (VALUES " + - "(0, 11), (1, 12), (2, 13), " + - "(3, 14), (4, 15), (5, 16), " + - "(6, 17), (7, 18), (8, 19)" + - " ) t (col0, col1) "; - assertUpdate(createTable, 9); - assertTrue(queryRunner.tableExists(getSession(), "test_bucket_hidden_column")); - - TableMetadata tableMetadata = getTableMetadata(catalog, TPCH_SCHEMA, "test_bucket_hidden_column"); - assertEquals(tableMetadata.getMetadata().getProperties().get(BUCKETED_BY_PROPERTY), ImmutableList.of("col0")); - assertEquals(tableMetadata.getMetadata().getProperties().get(BUCKET_COUNT_PROPERTY), 2); - - List columnNames = ImmutableList.of("col0", "col1", PATH_COLUMN_NAME, BUCKET_COLUMN_NAME); - List columnMetadatas = tableMetadata.getColumns(); - assertEquals(columnMetadatas.size(), columnNames.size()); - for (int i = 0; i < columnMetadatas.size(); i++) { - ColumnMetadata columnMetadata = columnMetadatas.get(i); - assertEquals(columnMetadata.getName(), columnNames.get(i)); - if (columnMetadata.getName().equals(BUCKET_COLUMN_NAME)) { - // $bucket_number should be hidden column - assertTrue(columnMetadata.isHidden()); - } - } - assertEquals(getBucketCount("test_bucket_hidden_column"), 2); - - MaterializedResult results = computeActual(format("SELECT *, \"%1$s\" FROM test_bucket_hidden_column WHERE \"%1$s\" = 1", - BUCKET_COLUMN_NAME)); - for (int i = 0; i < results.getRowCount(); i++) { - MaterializedRow row = results.getMaterializedRows().get(i); - int col0 = (int) row.getField(0); - int col1 = (int) row.getField(1); - int bucket = (int) row.getField(2); - - assertEquals(col1, col0 + 11); - assertTrue(col1 % 2 == 0); - - // Because Hive's hash function for integer n is h(n) = n. - assertEquals(bucket, col0 % 2); - } - assertEquals(results.getRowCount(), 4); - - assertUpdate("DROP TABLE test_bucket_hidden_column"); - assertFalse(queryRunner.tableExists(getSession(), "test_bucket_hidden_column")); - } - @Test public void testDeleteAndInsert() { @@ -1843,7 +1800,7 @@ private void verifyPartition(boolean hasPartition, TableMetadata tableMetadata, assertEquals(partitionByProperty, partitionKeys); for (ColumnMetadata columnMetadata : tableMetadata.getColumns()) { boolean partitionKey = partitionKeys.contains(columnMetadata.getName()); - assertEquals(columnMetadata.getExtraInfo(), columnExtraInfo(partitionKey)); + assertEquals(columnMetadata.getComment(), annotateColumnComment(Optional.empty(), partitionKey)); } } else { 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..c8d04484f306 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 @@ -52,7 +52,7 @@ public void testJsonRoundTrip() addresses, OptionalInt.empty(), true, - TupleDomain.all(), + TupleDomain.all(), ImmutableMap.of(1, HIVE_STRING)); String json = codec.toJson(expected); diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestJsonHiveHandles.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestJsonHiveHandles.java index eedae30ef5f4..29f66d8c691c 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestJsonHiveHandles.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/TestJsonHiveHandles.java @@ -33,7 +33,7 @@ @Test public class TestJsonHiveHandles { - private static final Map TABLE_HANDLE_AS_MAP = ImmutableMap.of( + private static final Map TABLE_HANDLE_AS_MAP = ImmutableMap.of( "clientId", "hive", "schemaName", "hive_schema", "tableName", "hive_table"); 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 48e3ce0c6146..6e7678b60c49 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 @@ -79,7 +79,6 @@ import java.lang.reflect.Field; import java.lang.reflect.Method; import java.util.List; -import java.util.OptionalInt; import java.util.Properties; import java.util.Random; import java.util.concurrent.ExecutorService; @@ -392,7 +391,6 @@ public ConnectorPageSource newPageSource() new Configuration(), SESSION, fileSplit.getPath(), - OptionalInt.empty(), fileSplit.getStart(), fileSplit.getLength(), schema, diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestPartitionOfflineException.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestPartitionOfflineException.java deleted file mode 100644 index a566763d3af4..000000000000 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestPartitionOfflineException.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.facebook.presto.hive; - -import com.facebook.presto.spi.SchemaTableName; -import org.testng.annotations.Test; - -import static org.testng.Assert.assertEquals; - -public class TestPartitionOfflineException -{ - @Test - public void testMessage() - throws Exception - { - assertMessage(new SchemaTableName("schema", "table"), "pk=1", false, "", "Table 'schema.table' partition 'pk=1' is offline"); - assertMessage(new SchemaTableName("schema", "table"), "pk=1", false, null, "Table 'schema.table' partition 'pk=1' is offline"); - assertMessage(new SchemaTableName("schema", "table"), "pk=1", true, "", "Table 'schema.table' partition 'pk=1' is offline for Presto"); - assertMessage(new SchemaTableName("schema", "table"), "pk=1", true, null, "Table 'schema.table' partition 'pk=1' is offline for Presto"); - assertMessage(new SchemaTableName("schema", "table"), "pk=1", false, "offline reason", "Table 'schema.table' partition 'pk=1' is offline: offline reason"); - assertMessage(new SchemaTableName("schema", "table"), "pk=1", true, "offline reason", "Table 'schema.table' partition 'pk=1' is offline for Presto: offline reason"); - } - - private static void assertMessage(SchemaTableName tableName, String partitionName, boolean forPresto, String offlineMessage, String expectedMessage) - { - PartitionOfflineException tableOfflineException = new PartitionOfflineException(tableName, partitionName, forPresto, offlineMessage); - assertEquals(tableOfflineException.getMessage(), expectedMessage); - } -} diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/TestTableOfflineException.java b/presto-hive/src/test/java/com/facebook/presto/hive/TestTableOfflineException.java deleted file mode 100644 index fcd4d462b169..000000000000 --- a/presto-hive/src/test/java/com/facebook/presto/hive/TestTableOfflineException.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.facebook.presto.hive; - -import com.facebook.presto.spi.SchemaTableName; -import org.testng.annotations.Test; - -import static org.testng.Assert.assertEquals; - -public class TestTableOfflineException -{ - @Test - public void testMessage() - throws Exception - { - assertMessage(new SchemaTableName("schema", "table"), false, "", "Table 'schema.table' is offline"); - assertMessage(new SchemaTableName("schema", "table"), false, null, "Table 'schema.table' is offline"); - assertMessage(new SchemaTableName("schema", "table"), true, "", "Table 'schema.table' is offline for Presto"); - assertMessage(new SchemaTableName("schema", "table"), true, null, "Table 'schema.table' is offline for Presto"); - assertMessage(new SchemaTableName("schema", "table"), false, "offline reason", "Table 'schema.table' is offline: offline reason"); - assertMessage(new SchemaTableName("schema", "table"), true, "offline reason", "Table 'schema.table' is offline for Presto: offline reason"); - } - - private static void assertMessage(SchemaTableName tableName, boolean forPresto, String offlineMessage, String expectedMessage) - { - TableOfflineException tableOfflineException = new TableOfflineException(tableName, forPresto, offlineMessage); - assertEquals(tableOfflineException.getMessage(), expectedMessage); - } -} diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/metastore/InMemoryHiveMetastore.java b/presto-hive/src/test/java/com/facebook/presto/hive/metastore/InMemoryHiveMetastore.java index 3a3657295e65..33ffda3738c9 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/metastore/InMemoryHiveMetastore.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/metastore/InMemoryHiveMetastore.java @@ -481,6 +481,11 @@ public synchronized void setTablePrivileges(String principalName, tablePrivileges.put(new PrincipalTableKey(principalName, principalType, tableName, databaseName), ImmutableSet.copyOf(privileges)); } + @Override + public void flushCache() + { + } + @Override public synchronized void grantTablePrivileges(String databaseName, String tableName, String grantee, Set privilegeGrantInfoSet) { diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/metastore/TestCachingHiveMetastore.java b/presto-hive/src/test/java/com/facebook/presto/hive/metastore/TestCachingHiveMetastore.java index 34c435f44764..a7b12329d277 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/metastore/TestCachingHiveMetastore.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/metastore/TestCachingHiveMetastore.java @@ -41,7 +41,7 @@ public class TestCachingHiveMetastore { private MockHiveMetastoreClient mockClient; - private CachingHiveMetastore metastore; + private ExtendedHiveMetastore metastore; private ThriftHiveMetastoreStats stats; @BeforeMethod @@ -56,8 +56,7 @@ public void setUp() new BridgingHiveMetastore(thriftHiveMetastore), executor, new Duration(5, TimeUnit.MINUTES), - new Duration(1, TimeUnit.MINUTES), - 1000); + new Duration(1, TimeUnit.MINUTES)); stats = thriftHiveMetastore.getStats(); } 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 7609ab81505e..ee2202e5e73d 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 @@ -17,8 +17,6 @@ import com.facebook.presto.hive.parquet.reader.ParquetReader; import com.facebook.presto.spi.block.Block; import com.facebook.presto.spi.type.Type; -import com.facebook.presto.spi.type.TypeManager; -import com.facebook.presto.type.TypeRegistry; import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.AbstractIterator; @@ -174,15 +172,16 @@ private static void assertFileContents(JobConf jobConf, FSDataInputStream inputStream = fileSystem.open(path); ParquetDataSource dataSource = new HdfsParquetDataSource(path, size, inputStream); - TypeManager typeManager = new TypeRegistry(); - ParquetReader parquetReader = new ParquetReader(fileSchema, fileSchema, parquetMetadata.getBlocks(), dataSource, typeManager); + ParquetReader parquetReader = new ParquetReader(fileSchema, + parquetMetadata.getBlocks(), + dataSource); assertEquals(parquetReader.getPosition(), 0); int rowsProcessed = 0; Iterator iterator = expectedValues.iterator(); for (int batchSize = parquetReader.nextBatch(); batchSize >= 0; batchSize = parquetReader.nextBatch()) { ColumnDescriptor columnDescriptor = fileSchema.getColumns().get(0); - Block block = parquetReader.readPrimitive(columnDescriptor, type); + Block block = parquetReader.readBlock(columnDescriptor, type); for (int i = 0; i < batchSize; i++) { assertTrue(iterator.hasNext()); Object expected = iterator.next(); diff --git a/presto-jmx/src/main/java/com/facebook/presto/connector/jmx/JmxConnectorConfig.java b/presto-jmx/src/main/java/com/facebook/presto/connector/jmx/JmxConnectorConfig.java index 5fe078afeff6..44df11c035ca 100644 --- a/presto-jmx/src/main/java/com/facebook/presto/connector/jmx/JmxConnectorConfig.java +++ b/presto-jmx/src/main/java/com/facebook/presto/connector/jmx/JmxConnectorConfig.java @@ -13,7 +13,6 @@ */ package com.facebook.presto.connector.jmx; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Splitter; import com.google.common.collect.ImmutableSet; import io.airlift.configuration.Config; @@ -24,8 +23,6 @@ import javax.validation.constraints.NotNull; import java.util.Set; -import java.util.regex.Pattern; -import java.util.stream.Collectors; import static java.util.concurrent.TimeUnit.SECONDS; @@ -44,19 +41,7 @@ public Set getDumpTables() @Config("jmx.dump-tables") public JmxConnectorConfig setDumpTables(String tableNames) { - this.dumpTables = Splitter.on(Pattern.compile("(? part.replace("\\,", ",")) // unescape all escaped commas - .collect(Collectors.toSet()); - return this; - } - - @VisibleForTesting - JmxConnectorConfig setDumpTables(Set tableNames) - { - this.dumpTables = ImmutableSet.copyOf(tableNames); + this.dumpTables = ImmutableSet.copyOf(Splitter.on(',').omitEmptyStrings().trimResults().split(tableNames)); return this; } diff --git a/presto-jmx/src/main/java/com/facebook/presto/connector/jmx/JmxRecordSetProvider.java b/presto-jmx/src/main/java/com/facebook/presto/connector/jmx/JmxRecordSetProvider.java index 30728734f102..a29226276426 100644 --- a/presto-jmx/src/main/java/com/facebook/presto/connector/jmx/JmxRecordSetProvider.java +++ b/presto-jmx/src/main/java/com/facebook/presto/connector/jmx/JmxRecordSetProvider.java @@ -191,7 +191,7 @@ private static Set getColumnNames(List columnHan return columnHandles.stream() .map(column -> checkType(column, JmxColumnHandle.class, "column")) .map(JmxColumnHandle::getColumnName) - .collect(Collectors.toSet()); + .collect(Collectors.toSet()); } private static List getColumnTypes(List columnHandles) @@ -199,7 +199,7 @@ private static List getColumnTypes(List columnHand return columnHandles.stream() .map(column -> checkType(column, JmxColumnHandle.class, "column")) .map(JmxColumnHandle::getColumnType) - .collect(Collectors.toList()); + .collect(Collectors.toList()); } private ImmutableMap> getAttributes(Set uniqueColumnNames, JmxTableHandle tableHandle) diff --git a/presto-jmx/src/test/java/com/facebook/presto/connector/jmx/TestJmxConnectorConfig.java b/presto-jmx/src/test/java/com/facebook/presto/connector/jmx/TestJmxConnectorConfig.java deleted file mode 100644 index d0d3b69f4180..000000000000 --- a/presto-jmx/src/test/java/com/facebook/presto/connector/jmx/TestJmxConnectorConfig.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.facebook.presto.connector.jmx; - -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import io.airlift.configuration.testing.ConfigAssertions; -import io.airlift.units.Duration; -import org.testng.annotations.Test; - -import java.util.Map; -import java.util.concurrent.TimeUnit; - -import static io.airlift.configuration.testing.ConfigAssertions.assertDeprecatedEquivalence; -import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; -import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; - -public class TestJmxConnectorConfig -{ - @Test - public void testDefaults() - { - assertRecordedDefaults(ConfigAssertions.recordDefaults(JmxConnectorConfig.class) - .setDumpTables("") - .setDumpPeriod(new Duration(10, TimeUnit.SECONDS)) - .setMaxEntries(24 * 60 * 60)); - } - - @Test - public void testExplicitPropertyMappings() - { - Map properties = new ImmutableMap.Builder() - .put("jmx.dump-tables", "table1,table\\,with\\,commas") - .put("jmx.dump-period", "1s") - .put("jmx.max-entries", "100") - .build(); - - JmxConnectorConfig expected = new JmxConnectorConfig() - .setDumpTables(ImmutableSet.of("table1", "table,with,commas")) - .setDumpPeriod(new Duration(1, TimeUnit.SECONDS)) - .setMaxEntries(100); - - assertFullMapping(properties, expected); - assertDeprecatedEquivalence(JmxConnectorConfig.class, properties); - } -} diff --git a/presto-jmx/src/test/java/com/facebook/presto/connector/jmx/TestJmxHistoricalData.java b/presto-jmx/src/test/java/com/facebook/presto/connector/jmx/TestJmxHistoricalData.java index c7e02ffa6b36..e05b61ddd802 100644 --- a/presto-jmx/src/test/java/com/facebook/presto/connector/jmx/TestJmxHistoricalData.java +++ b/presto-jmx/src/test/java/com/facebook/presto/connector/jmx/TestJmxHistoricalData.java @@ -36,14 +36,14 @@ public void testAddingRows() List secondColumn = ImmutableList.of(1); assertEquals(jmxHistoricalData.getRows(TABLE_NAME, bothColumns), ImmutableList.of()); - jmxHistoricalData.addRow(TABLE_NAME, ImmutableList.of(42, "ala")); + jmxHistoricalData.addRow(TABLE_NAME, ImmutableList.of(42, "ala")); assertEquals(jmxHistoricalData.getRows(TABLE_NAME, bothColumns), ImmutableList.of(ImmutableList.of(42, "ala"))); assertEquals(jmxHistoricalData.getRows(TABLE_NAME, secondColumn), ImmutableList.of(ImmutableList.of("ala"))); assertEquals(jmxHistoricalData.getRows(NOT_EXISTING_TABLE_NAME, bothColumns), ImmutableList.of()); - jmxHistoricalData.addRow(TABLE_NAME, ImmutableList.of(42, "ala")); - jmxHistoricalData.addRow(TABLE_NAME, ImmutableList.of(42, "ala")); - jmxHistoricalData.addRow(TABLE_NAME, ImmutableList.of(42, "ala")); + jmxHistoricalData.addRow(TABLE_NAME, ImmutableList.of(42, "ala")); + jmxHistoricalData.addRow(TABLE_NAME, ImmutableList.of(42, "ala")); + jmxHistoricalData.addRow(TABLE_NAME, ImmutableList.of(42, "ala")); assertEquals(jmxHistoricalData.getRows(TABLE_NAME, bothColumns).size(), MAX_ENTRIES); } @@ -56,8 +56,8 @@ public void testCaseInsensitive() assertEquals(jmxHistoricalData.getRows(TABLE_NAME, columns), ImmutableList.of()); assertEquals(jmxHistoricalData.getRows(TABLE_NAME.toUpperCase(), columns), ImmutableList.of()); - jmxHistoricalData.addRow(TABLE_NAME, ImmutableList.of(42)); - jmxHistoricalData.addRow(TABLE_NAME.toUpperCase(), ImmutableList.of(44)); + jmxHistoricalData.addRow(TABLE_NAME, ImmutableList.of(42)); + jmxHistoricalData.addRow(TABLE_NAME.toUpperCase(), ImmutableList.of(44)); assertEquals(jmxHistoricalData.getRows(TABLE_NAME, columns), ImmutableList.of( ImmutableList.of(42), ImmutableList.of(44))); diff --git a/presto-kafka/src/main/java/com/facebook/presto/kafka/KafkaTableDescriptionSupplier.java b/presto-kafka/src/main/java/com/facebook/presto/kafka/KafkaTableDescriptionSupplier.java index 03af3a19095f..fe3b22c1eaff 100644 --- a/presto-kafka/src/main/java/com/facebook/presto/kafka/KafkaTableDescriptionSupplier.java +++ b/presto-kafka/src/main/java/com/facebook/presto/kafka/KafkaTableDescriptionSupplier.java @@ -100,8 +100,8 @@ public Map get() builder.put(tableName, new KafkaTopicDescription(tableName.getTableName(), tableName.getSchemaName(), definedTable, - new KafkaTopicFieldGroup(DummyRowDecoder.NAME, ImmutableList.of()), - new KafkaTopicFieldGroup(DummyRowDecoder.NAME, ImmutableList.of()))); + new KafkaTopicFieldGroup(DummyRowDecoder.NAME, ImmutableList.of()), + new KafkaTopicFieldGroup(DummyRowDecoder.NAME, ImmutableList.of()))); } } diff --git a/presto-kafka/src/test/java/com/facebook/presto/kafka/util/CodecSupplier.java b/presto-kafka/src/test/java/com/facebook/presto/kafka/util/CodecSupplier.java index 05f7a035869f..893941fd52d4 100644 --- a/presto-kafka/src/test/java/com/facebook/presto/kafka/util/CodecSupplier.java +++ b/presto-kafka/src/test/java/com/facebook/presto/kafka/util/CodecSupplier.java @@ -16,6 +16,7 @@ import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.type.Type; import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; import com.fasterxml.jackson.databind.deser.std.FromStringDeserializer; import com.google.common.collect.ImmutableMap; import io.airlift.json.JsonCodec; @@ -38,7 +39,7 @@ public CodecSupplier(Class clazz, Metadata metadata) this.clazz = clazz; this.metadata = metadata; ObjectMapperProvider objectMapperProvider = new ObjectMapperProvider(); - objectMapperProvider.setJsonDeserializers(ImmutableMap.of(Type.class, new TypeDeserializer())); + objectMapperProvider.setJsonDeserializers(ImmutableMap., JsonDeserializer>of(Type.class, new TypeDeserializer())); this.codecFactory = new JsonCodecFactory(objectMapperProvider); } diff --git a/presto-local-file/src/main/java/com/facebook/presto/localfile/DataLocation.java b/presto-local-file/src/main/java/com/facebook/presto/localfile/DataLocation.java index 7e333e30ea49..88bffa530425 100644 --- a/presto-local-file/src/main/java/com/facebook/presto/localfile/DataLocation.java +++ b/presto-local-file/src/main/java/com/facebook/presto/localfile/DataLocation.java @@ -83,7 +83,7 @@ public List files() checkState(location.isDirectory(), "location %s is not a directory", location); try (DirectoryStream paths = newDirectoryStream(location.toPath(), pattern.get())) { - ImmutableList.Builder builder = ImmutableList.builder(); + ImmutableList.Builder builder = ImmutableList.builder(); for (Path path : paths) { builder.add(path.toFile()); } diff --git a/presto-local-file/src/test/java/com/facebook/presto/localfile/MetadataUtil.java b/presto-local-file/src/test/java/com/facebook/presto/localfile/MetadataUtil.java index 6d13e3e72562..e71d17b92601 100644 --- a/presto-local-file/src/test/java/com/facebook/presto/localfile/MetadataUtil.java +++ b/presto-local-file/src/test/java/com/facebook/presto/localfile/MetadataUtil.java @@ -16,6 +16,7 @@ import com.facebook.presto.spi.type.StandardTypes; import com.facebook.presto.spi.type.Type; import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; import com.fasterxml.jackson.databind.deser.std.FromStringDeserializer; import com.google.common.collect.ImmutableMap; import io.airlift.json.JsonCodec; @@ -43,7 +44,7 @@ private MetadataUtil() static { ObjectMapperProvider objectMapperProvider = new ObjectMapperProvider(); - objectMapperProvider.setJsonDeserializers(ImmutableMap.of(Type.class, new TestingTypeDeserializer())); + objectMapperProvider.setJsonDeserializers(ImmutableMap., JsonDeserializer>of(Type.class, new TestingTypeDeserializer())); JsonCodecFactory codecFactory = new JsonCodecFactory(objectMapperProvider); COLUMN_CODEC = codecFactory.jsonCodec(LocalFileColumnHandle.class); TABLE_CODEC = codecFactory.jsonCodec(LocalFileTableHandle.class); diff --git a/presto-main/src/main/java/com/facebook/presto/Session.java b/presto-main/src/main/java/com/facebook/presto/Session.java index 81f52947e46d..a9896cb0d4f5 100644 --- a/presto-main/src/main/java/com/facebook/presto/Session.java +++ b/presto-main/src/main/java/com/facebook/presto/Session.java @@ -85,7 +85,7 @@ public Session( this.queryId = requireNonNull(queryId, "queryId is null"); this.transactionId = requireNonNull(transactionId, "transactionId is null"); this.clientTransactionSupport = clientTransactionSupport; - this.identity = requireNonNull(identity, "identity is null"); + this.identity = identity; this.source = requireNonNull(source, "source is null"); this.catalog = requireNonNull(catalog, "catalog is null"); this.schema = requireNonNull(schema, "schema is null"); diff --git a/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java b/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java index 2464b59aff3d..252d63e20188 100644 --- a/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java +++ b/presto-main/src/main/java/com/facebook/presto/SystemSessionProperties.java @@ -64,7 +64,6 @@ public final class SystemSessionProperties public static final String SPILL_ENABLED = "spill_enabled"; public static final String OPERATOR_MEMORY_LIMIT_BEFORE_SPILL = "operator_memory_limit_before_spill"; public static final String OPTIMIZE_DISTINCT_AGGREGATIONS = "optimize_mixed_distinct_aggregations"; - public static final String LEGACY_ORDER_BY = "legacy_order_by"; private final List> sessionProperties; @@ -254,11 +253,6 @@ public SystemSessionProperties( OPTIMIZE_DISTINCT_AGGREGATIONS, "Optimize mixed non-distinct and distinct aggregations", featuresConfig.isOptimizeMixedDistinctAggregations(), - false), - booleanSessionProperty( - LEGACY_ORDER_BY, - "Use legacy rules for column resolution in ORDER BY clause", - featuresConfig.isLegacyOrderBy(), false)); } @@ -400,9 +394,4 @@ public static boolean isOptimizeDistinctAggregationEnabled(Session session) { return session.getSystemProperty(OPTIMIZE_DISTINCT_AGGREGATIONS, Boolean.class); } - - public static boolean isLegacyOrderByEnabled(Session session) - { - return session.getSystemProperty(LEGACY_ORDER_BY, Boolean.class); - } } diff --git a/presto-main/src/main/java/com/facebook/presto/connector/informationSchema/InformationSchemaMetadata.java b/presto-main/src/main/java/com/facebook/presto/connector/informationSchema/InformationSchemaMetadata.java index fcae6ac3b4a3..724090c8e992 100644 --- a/presto-main/src/main/java/com/facebook/presto/connector/informationSchema/InformationSchemaMetadata.java +++ b/presto-main/src/main/java/com/facebook/presto/connector/informationSchema/InformationSchemaMetadata.java @@ -70,7 +70,6 @@ public class InformationSchemaMetadata .column("is_nullable", createUnboundedVarcharType()) .column("data_type", createUnboundedVarcharType()) .column("comment", createUnboundedVarcharType()) - .column("extra_info", createUnboundedVarcharType()) .build()) .table(tableMetadataBuilder(TABLE_TABLES) .column("table_catalog", createUnboundedVarcharType()) diff --git a/presto-main/src/main/java/com/facebook/presto/connector/informationSchema/InformationSchemaPageSourceProvider.java b/presto-main/src/main/java/com/facebook/presto/connector/informationSchema/InformationSchemaPageSourceProvider.java index 723456e0c245..56400c21e862 100644 --- a/presto-main/src/main/java/com/facebook/presto/connector/informationSchema/InformationSchemaPageSourceProvider.java +++ b/presto-main/src/main/java/com/facebook/presto/connector/informationSchema/InformationSchemaPageSourceProvider.java @@ -169,8 +169,7 @@ private InternalTable buildColumns(Session session, String catalogName, Map layouts = metadata.getLayouts(session, tableHandle.get(), Constraint.alwaysTrue(), Optional.empty()); + List layouts = metadata.getLayouts(session, tableHandle.get(), Constraint.alwaysTrue(), Optional.empty()); if (layouts.size() == 1) { Map columnHandles = ImmutableBiMap.copyOf(metadata.getColumnHandles(session, tableHandle.get())).inverse(); diff --git a/presto-main/src/main/java/com/facebook/presto/event/query/QueryMonitor.java b/presto-main/src/main/java/com/facebook/presto/event/query/QueryMonitor.java index ddcb49855ef5..2d43613dc3a4 100644 --- a/presto-main/src/main/java/com/facebook/presto/event/query/QueryMonitor.java +++ b/presto-main/src/main/java/com/facebook/presto/event/query/QueryMonitor.java @@ -226,7 +226,7 @@ private static Map mergeSessionAndCatalogProperties(QueryInfo qu return mergedProperties.build(); } - private static void logQueryTimeline(QueryInfo queryInfo) + private void logQueryTimeline(QueryInfo queryInfo) { try { QueryStats queryStats = queryInfo.getQueryStats(); diff --git a/presto-main/src/main/java/com/facebook/presto/execution/CreateViewTask.java b/presto-main/src/main/java/com/facebook/presto/execution/CreateViewTask.java index 754ebdac2f7d..d48df09549e1 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/CreateViewTask.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/CreateViewTask.java @@ -21,6 +21,7 @@ import com.facebook.presto.sql.analyzer.Analysis; import com.facebook.presto.sql.analyzer.Analyzer; import com.facebook.presto.sql.analyzer.FeaturesConfig; +import com.facebook.presto.sql.analyzer.QueryExplainer; import com.facebook.presto.sql.parser.SqlParser; import com.facebook.presto.sql.tree.CreateView; import com.facebook.presto.sql.tree.Expression; @@ -51,6 +52,7 @@ public class CreateViewTask public CreateViewTask( JsonCodec codec, SqlParser sqlParser, + AccessControl accessControl, FeaturesConfig featuresConfig) { this.codec = requireNonNull(codec, "codec is null"); @@ -82,7 +84,7 @@ public CompletableFuture execute(CreateView statement, TransactionManager tra Analysis analysis = analyzeStatement(statement, session, metadata, accessControl, parameters); - List columns = analysis.getOutputDescriptor(statement.getQuery()) + List columns = analysis.getOutputDescriptor() .getVisibleFields().stream() .map(field -> new ViewColumn(field.getName().get(), field.getType())) .collect(toImmutableList()); @@ -96,7 +98,7 @@ public CompletableFuture execute(CreateView statement, TransactionManager tra private Analysis analyzeStatement(Statement statement, Session session, Metadata metadata, AccessControl accessControl, List parameters) { - Analyzer analyzer = new Analyzer(session, metadata, sqlParser, accessControl, Optional.empty(), parameters); + Analyzer analyzer = new Analyzer(session, metadata, sqlParser, accessControl, Optional.empty(), parameters); return analyzer.analyze(statement); } } diff --git a/presto-main/src/main/java/com/facebook/presto/execution/FutureStateChange.java b/presto-main/src/main/java/com/facebook/presto/execution/FutureStateChange.java deleted file mode 100644 index 88edf4014e3c..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/execution/FutureStateChange.java +++ /dev/null @@ -1,76 +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.execution; - -import com.google.common.collect.ImmutableSet; - -import javax.annotation.concurrent.GuardedBy; -import javax.annotation.concurrent.ThreadSafe; - -import java.util.HashSet; -import java.util.Set; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Executor; - -import static com.google.common.util.concurrent.MoreExecutors.directExecutor; -import static java.util.Objects.requireNonNull; - -@ThreadSafe -public class FutureStateChange -{ - // Use a separate future for each listener so canceled listeners can be removed - @GuardedBy("listeners") - private final Set> listeners = new HashSet<>(); - - public CompletableFuture createNewListener() - { - CompletableFuture listener = new CompletableFuture<>(); - synchronized (listeners) { - listeners.add(listener); - } - - // remove the listener when the future completes - listener.whenComplete((t, throwable) -> { - synchronized (listeners) { - listeners.remove(listener); - } - }); - - return listener; - } - - public void complete(T newState) - { - fireStateChange(newState, directExecutor()); - } - - public void complete(T newState, Executor executor) - { - fireStateChange(newState, executor); - } - - private void fireStateChange(T newState, Executor executor) - { - requireNonNull(executor, "executor is null"); - Set> futures; - synchronized (listeners) { - futures = ImmutableSet.copyOf(listeners); - listeners.clear(); - } - - for (CompletableFuture future : futures) { - executor.execute(() -> future.complete(newState)); - } - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/execution/QueryStateMachine.java b/presto-main/src/main/java/com/facebook/presto/execution/QueryStateMachine.java index f4c87547be87..f7381354a621 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/QueryStateMachine.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/QueryStateMachine.java @@ -42,6 +42,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.CompletionException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicBoolean; @@ -62,7 +63,6 @@ import static com.facebook.presto.spi.StandardErrorCode.USER_CANCELED; import static com.facebook.presto.util.Failures.toFailure; import static com.google.common.base.Preconditions.checkArgument; -import static io.airlift.concurrent.MoreFutures.unwrapCompletionException; import static io.airlift.units.DataSize.succinctBytes; import static io.airlift.units.Duration.succinctNanos; import static java.util.Objects.requireNonNull; @@ -749,4 +749,13 @@ private Duration nanosSince(long start) { return succinctNanos(tickerNanos() - start); } + + // TODO: move to Airlift MoreFutures + private static Throwable unwrapCompletionException(Throwable throwable) + { + if (throwable instanceof CompletionException) { + return throwable.getCause(); + } + return throwable; + } } diff --git a/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryExecution.java b/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryExecution.java index 4fe54795efd9..3c88685431b1 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryExecution.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryExecution.java @@ -300,7 +300,7 @@ private PlanRoot doAnalyzeQuery() stateMachine.setOutput(output); // fragment the plan - SubPlan subplan = PlanFragmenter.createSubPlans(stateMachine.getSession(), metadata, plan); + SubPlan subplan = new PlanFragmenter().createSubPlans(stateMachine.getSession(), metadata, plan); // record analysis time stateMachine.recordAnalysisTime(analysisStart); diff --git a/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryQueueManager.java b/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryQueueManager.java index d7ccbae0d03c..113ed4b17c45 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryQueueManager.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/SqlQueryQueueManager.java @@ -67,7 +67,7 @@ public void submit(Statement statement, QueryExecution queryExecution, Executor if (!queue.reserve(queryExecution)) { // Reject query if we couldn't acquire a permit to enter the queue. // The permits will be released when this query fails. - queryExecution.fail(new PrestoException(QUERY_QUEUE_FULL, "Too many queued queries")); + queryExecution.fail(new PrestoException(QUERY_QUEUE_FULL, "Too many queued queries!")); return; } } diff --git a/presto-main/src/main/java/com/facebook/presto/execution/StartTransactionTask.java b/presto-main/src/main/java/com/facebook/presto/execution/StartTransactionTask.java index 7a2ec8d117e5..b8fbb08e9c2c 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/StartTransactionTask.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/StartTransactionTask.java @@ -77,7 +77,7 @@ public boolean isTransactionControl() return true; } - private static Optional extractIsolationLevel(StartTransaction startTransaction) + private Optional extractIsolationLevel(StartTransaction startTransaction) { if (startTransaction.getTransactionModes().stream() .filter(Isolation.class::isInstance) @@ -93,7 +93,7 @@ private static Optional extractIsolationLevel(StartTransaction s .findFirst(); } - private static Optional extractReadOnly(StartTransaction startTransaction) + private Optional extractReadOnly(StartTransaction startTransaction) { if (startTransaction.getTransactionModes().stream() .filter(TransactionAccessMode.class::isInstance) diff --git a/presto-main/src/main/java/com/facebook/presto/execution/StateMachine.java b/presto-main/src/main/java/com/facebook/presto/execution/StateMachine.java index 3a7064395e89..658dd9cc20bf 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/StateMachine.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/StateMachine.java @@ -24,6 +24,7 @@ import javax.annotation.concurrent.ThreadSafe; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -329,4 +330,39 @@ public String toString() { return get().toString(); } + + private static class FutureStateChange + { + // Use a separate future for each listener so canceled listeners can be removed + @GuardedBy("this") + private final Set> listeners = new HashSet<>(); + + public synchronized CompletableFuture createNewListener() + { + CompletableFuture listener = new CompletableFuture<>(); + listeners.add(listener); + + // remove the listener when the future completes + listener.whenComplete((t, throwable) -> { + synchronized (FutureStateChange.this) { + listeners.remove(listener); + } + }); + + return listener; + } + + public void complete(T newState) + { + Set> futures; + synchronized (this) { + futures = ImmutableSet.copyOf(listeners); + listeners.clear(); + } + + for (CompletableFuture future : futures) { + future.complete(newState); + } + } + } } diff --git a/presto-main/src/main/java/com/facebook/presto/execution/TaskInfo.java b/presto-main/src/main/java/com/facebook/presto/execution/TaskInfo.java index e4ef5c8da181..085d02b11964 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/TaskInfo.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/TaskInfo.java @@ -108,7 +108,7 @@ public boolean isComplete() public TaskInfo summarize() { - return new TaskInfo(taskStatus, lastHeartbeat, outputBuffers.summarize(), noMoreSplits, stats.summarize(), needsPlan, complete); + return new TaskInfo(taskStatus, lastHeartbeat, outputBuffers, noMoreSplits, stats.summarize(), needsPlan, complete); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/execution/buffer/BufferResult.java b/presto-main/src/main/java/com/facebook/presto/execution/buffer/BufferResult.java index 434a7100b829..6f06978ab14a 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/buffer/BufferResult.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/buffer/BufferResult.java @@ -28,7 +28,7 @@ public class BufferResult { public static BufferResult emptyResults(String taskInstanceId, long token, boolean bufferComplete) { - return new BufferResult(taskInstanceId, token, token, bufferComplete, ImmutableList.of()); + return new BufferResult(taskInstanceId, token, token, bufferComplete, ImmutableList.of()); } private final String taskInstanceId; diff --git a/presto-main/src/main/java/com/facebook/presto/execution/buffer/OutputBufferInfo.java b/presto-main/src/main/java/com/facebook/presto/execution/buffer/OutputBufferInfo.java index aa44d3c00764..8afcdf31668c 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/buffer/OutputBufferInfo.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/buffer/OutputBufferInfo.java @@ -111,11 +111,6 @@ public long getTotalPagesSent() return totalPagesSent; } - public OutputBufferInfo summarize() - { - return new OutputBufferInfo(type, state, canAddBuffers, canAddPages, totalBufferedBytes, totalBufferedPages, totalRowsSent, totalPagesSent, ImmutableList.of()); - } - @Override public boolean equals(Object o) { diff --git a/presto-main/src/main/java/com/facebook/presto/execution/resourceGroups/InternalResourceGroup.java b/presto-main/src/main/java/com/facebook/presto/execution/resourceGroups/InternalResourceGroup.java index e34c9df55595..5af89f2a0792 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/resourceGroups/InternalResourceGroup.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/resourceGroups/InternalResourceGroup.java @@ -412,7 +412,7 @@ public void run(QueryExecution query) group = group.parent.get(); } if (!canQueue && !canRun) { - query.fail(new PrestoException(QUERY_QUEUE_FULL, format("Too many queued queries for \"%s\"", id))); + query.fail(new PrestoException(QUERY_QUEUE_FULL, format("Too many queued queries for \"%s\"!", id))); return; } if (canRun) { diff --git a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/NodeScheduler.java b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/NodeScheduler.java index d187cd82da98..8c349ee57d28 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/scheduler/NodeScheduler.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/scheduler/NodeScheduler.java @@ -43,7 +43,6 @@ import java.util.LinkedHashSet; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -307,12 +306,8 @@ public static CompletableFuture toWhenHasSplitQueueSpaceFuture(Set bloc List> blockedFutures = blockedNodes.stream() .map(Node::getNodeIdentifier) .map(nodeToTaskMap::get) - .filter(Objects::nonNull) .map(remoteTask -> remoteTask.whenSplitQueueHasSpace(spaceThreshold)) .collect(toImmutableList()); - if (blockedFutures.isEmpty()) { - return completedFuture(null); - } return firstCompletedFuture(blockedFutures, true); } diff --git a/presto-main/src/main/java/com/facebook/presto/memory/QueryContext.java b/presto-main/src/main/java/com/facebook/presto/memory/QueryContext.java index 31a38024b2c0..21423fea987d 100644 --- a/presto-main/src/main/java/com/facebook/presto/memory/QueryContext.java +++ b/presto-main/src/main/java/com/facebook/presto/memory/QueryContext.java @@ -30,17 +30,13 @@ import java.util.concurrent.Executor; import static com.facebook.presto.ExceededMemoryLimitException.exceededLocalLimit; -import static com.facebook.presto.operator.Operator.NOT_BLOCKED; import static com.google.common.base.Preconditions.checkArgument; -import static io.airlift.units.DataSize.Unit.MEGABYTE; import static io.airlift.units.DataSize.succinctBytes; import static java.util.Objects.requireNonNull; @ThreadSafe public class QueryContext { - private static final long GUARANTEED_MEMORY = new DataSize(1, MEGABYTE).toBytes(); - private final QueryId queryId; private final Executor executor; private final List taskContexts = new CopyOnWriteArrayList<>(); @@ -85,10 +81,6 @@ public synchronized ListenableFuture reserveMemory(long bytes) } ListenableFuture future = memoryPool.reserve(queryId, bytes); reserved += bytes; - // Never block queries using a trivial amount of memory - if (reserved < GUARANTEED_MEMORY) { - return NOT_BLOCKED; - } return future; } diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/FunctionRegistry.java b/presto-main/src/main/java/com/facebook/presto/metadata/FunctionRegistry.java index 6a977f62d567..44647f86ff69 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/FunctionRegistry.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/FunctionRegistry.java @@ -76,7 +76,6 @@ import com.facebook.presto.operator.scalar.ColorFunctions; import com.facebook.presto.operator.scalar.CombineHashFunction; import com.facebook.presto.operator.scalar.DateTimeFunctions; -import com.facebook.presto.operator.scalar.EmptyMapConstructor; import com.facebook.presto.operator.scalar.FailureFunction; import com.facebook.presto.operator.scalar.HyperLogLogFunctions; import com.facebook.presto.operator.scalar.JoniRegexpCasts; @@ -121,7 +120,6 @@ import com.facebook.presto.spi.type.TypeSignature; import com.facebook.presto.spi.type.VarcharType; import com.facebook.presto.sql.analyzer.FeaturesConfig; -import com.facebook.presto.sql.analyzer.TypeSignatureProvider; import com.facebook.presto.sql.tree.QualifiedName; import com.facebook.presto.type.BigintOperators; import com.facebook.presto.type.BooleanOperators; @@ -156,6 +154,7 @@ import com.google.common.collect.ImmutableListMultimap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; import com.google.common.collect.Multimap; import com.google.common.collect.Multimaps; import com.google.common.collect.Ordering; @@ -199,16 +198,13 @@ import static com.facebook.presto.operator.aggregation.MinNAggregationFunction.MIN_N_AGGREGATION; import static com.facebook.presto.operator.aggregation.MultimapAggregationFunction.MULTIMAP_AGG; import static com.facebook.presto.operator.scalar.ArrayConstructor.ARRAY_CONSTRUCTOR; -import static com.facebook.presto.operator.scalar.ArrayFilterFunction.ARRAY_FILTER_FUNCTION; import static com.facebook.presto.operator.scalar.ArrayFlattenFunction.ARRAY_FLATTEN_FUNCTION; import static com.facebook.presto.operator.scalar.ArrayJoin.ARRAY_JOIN; import static com.facebook.presto.operator.scalar.ArrayJoin.ARRAY_JOIN_WITH_NULL_REPLACEMENT; -import static com.facebook.presto.operator.scalar.ArrayReduceFunction.ARRAY_REDUCE_FUNCTION; import static com.facebook.presto.operator.scalar.ArraySubscriptOperator.ARRAY_SUBSCRIPT; import static com.facebook.presto.operator.scalar.ArrayToArrayCast.ARRAY_TO_ARRAY_CAST; import static com.facebook.presto.operator.scalar.ArrayToElementConcatFunction.ARRAY_TO_ELEMENT_CONCAT_FUNCTION; import static com.facebook.presto.operator.scalar.ArrayToJsonCast.ARRAY_TO_JSON; -import static com.facebook.presto.operator.scalar.ArrayTransformFunction.ARRAY_TRANSFORM_FUNCTION; import static com.facebook.presto.operator.scalar.CastFromUnknownOperator.CAST_FROM_UNKNOWN; import static com.facebook.presto.operator.scalar.ConcatFunction.CONCAT; import static com.facebook.presto.operator.scalar.ElementToArrayConcatFunction.ELEMENT_TO_ARRAY_CONCAT_FUNCTION; @@ -219,7 +215,6 @@ import static com.facebook.presto.operator.scalar.Least.LEAST; import static com.facebook.presto.operator.scalar.MapConstructor.MAP_CONSTRUCTOR; import static com.facebook.presto.operator.scalar.MapElementAtFunction.MAP_ELEMENT_AT; -import static com.facebook.presto.operator.scalar.MapFilterFunction.MAP_FILTER_FUNCTION; import static com.facebook.presto.operator.scalar.MapHashCodeOperator.MAP_HASH_CODE; import static com.facebook.presto.operator.scalar.MapSubscriptOperator.MAP_SUBSCRIPT; import static com.facebook.presto.operator.scalar.MapToJsonCast.MAP_TO_JSON; @@ -247,8 +242,6 @@ import static com.facebook.presto.spi.type.DoubleType.DOUBLE; import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature; import static com.facebook.presto.spi.type.VarbinaryType.VARBINARY; -import static com.facebook.presto.sql.analyzer.TypeSignatureProvider.fromTypeSignatures; -import static com.facebook.presto.sql.analyzer.TypeSignatureProvider.fromTypes; import static com.facebook.presto.type.DecimalCasts.BIGINT_TO_DECIMAL_CAST; import static com.facebook.presto.type.DecimalCasts.BOOLEAN_TO_DECIMAL_CAST; import static com.facebook.presto.type.DecimalCasts.DECIMAL_TO_BIGINT_CAST; @@ -280,17 +273,7 @@ import static com.facebook.presto.type.DecimalOperators.DECIMAL_MODULUS_OPERATOR; import static com.facebook.presto.type.DecimalOperators.DECIMAL_MULTIPLY_OPERATOR; import static com.facebook.presto.type.DecimalOperators.DECIMAL_SUBTRACT_OPERATOR; -import static com.facebook.presto.type.DecimalSaturatedFloorCasts.BIGINT_TO_DECIMAL_SATURATED_FLOOR_CAST; -import static com.facebook.presto.type.DecimalSaturatedFloorCasts.DECIMAL_TO_BIGINT_SATURATED_FLOOR_CAST; import static com.facebook.presto.type.DecimalSaturatedFloorCasts.DECIMAL_TO_DECIMAL_SATURATED_FLOOR_CAST; -import static com.facebook.presto.type.DecimalSaturatedFloorCasts.DECIMAL_TO_INTEGER_SATURATED_FLOOR_CAST; -import static com.facebook.presto.type.DecimalSaturatedFloorCasts.DECIMAL_TO_SMALLINT_SATURATED_FLOOR_CAST; -import static com.facebook.presto.type.DecimalSaturatedFloorCasts.DECIMAL_TO_TINYINT_SATURATED_FLOOR_CAST; -import static com.facebook.presto.type.DecimalSaturatedFloorCasts.DOUBLE_TO_DECIMAL_SATURATED_FLOOR_CAST; -import static com.facebook.presto.type.DecimalSaturatedFloorCasts.INTEGER_TO_DECIMAL_SATURATED_FLOOR_CAST; -import static com.facebook.presto.type.DecimalSaturatedFloorCasts.REAL_TO_DECIMAL_SATURATED_FLOOR_CAST; -import static com.facebook.presto.type.DecimalSaturatedFloorCasts.SMALLINT_TO_DECIMAL_SATURATED_FLOOR_CAST; -import static com.facebook.presto.type.DecimalSaturatedFloorCasts.TINYINT_TO_DECIMAL_SATURATED_FLOOR_CAST; import static com.facebook.presto.type.DecimalToDecimalCasts.DECIMAL_TO_DECIMAL_CAST; import static com.facebook.presto.type.TypeUtils.resolveTypes; import static com.facebook.presto.type.UnknownType.UNKNOWN; @@ -311,7 +294,7 @@ public class FunctionRegistry private static final String OPERATOR_PREFIX = "$operator$"; // hack: java classes for types that can be used with magic literals - private static final Set> SUPPORTED_LITERAL_TYPES = ImmutableSet.of(long.class, double.class, Slice.class, boolean.class); + private static final Set> SUPPORTED_LITERAL_TYPES = ImmutableSet.>of(long.class, double.class, Slice.class, boolean.class); private final TypeManager typeManager; private final LoadingCache specializedFunctionKeyCache; @@ -500,7 +483,6 @@ public WindowFunctionSupplier load(SpecializedFunctionKey key) .scalar(MapCardinalityFunction.class) .scalar(MapConcatFunction.class) .scalar(MapToMapCast.class) - .scalars(EmptyMapConstructor.class) .scalar(TypeOfFunction.class) .functions(ZIP_FUNCTIONS) .functions(ARRAY_JOIN, ARRAY_JOIN_WITH_NULL_REPLACEMENT) @@ -519,12 +501,7 @@ public WindowFunctionSupplier load(SpecializedFunctionKey key) .functions(DECIMAL_EQUAL_OPERATOR, DECIMAL_NOT_EQUAL_OPERATOR) .functions(DECIMAL_LESS_THAN_OPERATOR, DECIMAL_LESS_THAN_OR_EQUAL_OPERATOR) .functions(DECIMAL_GREATER_THAN_OPERATOR, DECIMAL_GREATER_THAN_OR_EQUAL_OPERATOR) - .function(DECIMAL_TO_DECIMAL_SATURATED_FLOOR_CAST) - .functions(DOUBLE_TO_DECIMAL_SATURATED_FLOOR_CAST, REAL_TO_DECIMAL_SATURATED_FLOOR_CAST) - .functions(DECIMAL_TO_BIGINT_SATURATED_FLOOR_CAST, BIGINT_TO_DECIMAL_SATURATED_FLOOR_CAST) - .functions(DECIMAL_TO_INTEGER_SATURATED_FLOOR_CAST, INTEGER_TO_DECIMAL_SATURATED_FLOOR_CAST) - .functions(DECIMAL_TO_SMALLINT_SATURATED_FLOOR_CAST, SMALLINT_TO_DECIMAL_SATURATED_FLOOR_CAST) - .functions(DECIMAL_TO_TINYINT_SATURATED_FLOOR_CAST, TINYINT_TO_DECIMAL_SATURATED_FLOOR_CAST) + .functions(DECIMAL_TO_DECIMAL_SATURATED_FLOOR_CAST) .function(DECIMAL_BETWEEN_OPERATOR) .function(DECIMAL_DISTINCT_FROM_OPERATOR) .function(HISTOGRAM) @@ -547,8 +524,6 @@ public WindowFunctionSupplier load(SpecializedFunctionKey key) .function(DECIMAL_MOD_FUNCTION) .functions(DECIMAL_ROUND_FUNCTIONS) .function(DECIMAL_TRUNCATE_FUNCTION) - .functions(ARRAY_TRANSFORM_FUNCTION, ARRAY_FILTER_FUNCTION, ARRAY_REDUCE_FUNCTION) - .functions(MAP_FILTER_FUNCTION) .function(TRY_CAST); builder.function(new ArrayAggregationFunction(featuresConfig.isLegacyArrayAgg())); @@ -587,14 +562,16 @@ public boolean isAggregationFunction(QualifiedName name) return Iterables.any(functions.get(name), function -> function.getSignature().getKind() == AGGREGATE); } - public Signature resolveFunction(QualifiedName name, List parameterTypes) + public Signature resolveFunction(QualifiedName name, List parameterTypes) { Collection allCandidates = functions.get(name); List exactCandidates = allCandidates.stream() .filter(function -> function.getSignature().getTypeVariableConstraints().isEmpty()) .collect(Collectors.toList()); - Optional match = matchFunctionExact(exactCandidates, parameterTypes); + List resolvedTypes = resolveTypes(parameterTypes, typeManager); + + Optional match = matchFunctionExact(exactCandidates, resolvedTypes); if (match.isPresent()) { return match.get(); } @@ -603,12 +580,12 @@ public Signature resolveFunction(QualifiedName name, List .filter(function -> !function.getSignature().getTypeVariableConstraints().isEmpty()) .collect(Collectors.toList()); - match = matchFunctionExact(genericCandidates, parameterTypes); + match = matchFunctionExact(genericCandidates, resolvedTypes); if (match.isPresent()) { return match.get(); } - match = matchFunctionWithCoercion(allCandidates, parameterTypes); + match = matchFunctionWithCoercion(allCandidates, resolvedTypes); if (match.isPresent()) { return match.get(); } @@ -637,7 +614,7 @@ public Signature resolveFunction(QualifiedName name, List // verify we have one parameter of the proper type checkArgument(parameterTypes.size() == 1, "Expected one argument to literal function, but got %s", parameterTypes); - Type parameterType = typeManager.getType(parameterTypes.get(0).getTypeSignature()); + Type parameterType = typeManager.getType(parameterTypes.get(0)); requireNonNull(parameterType, format("Type %s not found", parameterTypes.get(0))); return getMagicLiteralFunctionSignature(type); @@ -646,17 +623,17 @@ public Signature resolveFunction(QualifiedName name, List throw new PrestoException(FUNCTION_NOT_FOUND, message); } - private Optional matchFunctionExact(List candidates, List actualParameters) + private Optional matchFunctionExact(List candidates, List actualParameters) { return matchFunction(candidates, actualParameters, false); } - private Optional matchFunctionWithCoercion(Collection candidates, List actualParameters) + private Optional matchFunctionWithCoercion(Collection candidates, List actualParameters) { return matchFunction(candidates, actualParameters, true); } - private Optional matchFunction(Collection candidates, List parameters, boolean coercionAllowed) + private Optional matchFunction(Collection candidates, List parameters, boolean coercionAllowed) { List applicableFunctions = identifyApplicableFunctions(candidates, parameters, coercionAllowed); if (applicableFunctions.isEmpty()) { @@ -683,7 +660,7 @@ private Optional matchFunction(Collection candidates, Li throw new PrestoException(AMBIGUOUS_FUNCTION_CALL, errorMessageBuilder.toString()); } - private List identifyApplicableFunctions(Collection candidates, List actualParameters, boolean allowCoercion) + private List identifyApplicableFunctions(Collection candidates, List actualParameters, boolean allowCoercion) { ImmutableList.Builder applicableFunctions = ImmutableList.builder(); for (SqlFunction function : candidates) { @@ -697,47 +674,29 @@ private List identifyApplicableFunctions(Collection selectMostSpecificFunctions(List applicableFunctions, List parameters) + private List selectMostSpecificFunctions(List applicableFunctions, List parameters) { - checkArgument(!applicableFunctions.isEmpty()); - List mostSpecificFunctions = selectMostSpecificFunctions(applicableFunctions); - if (mostSpecificFunctions.size() <= 1) { - return mostSpecificFunctions; - } - - Optional> optionalParameterTypes = toTypes(parameters, typeManager); - if (!optionalParameterTypes.isPresent()) { - // give up and return all remaining matches - return mostSpecificFunctions; - } - - List parameterTypes = optionalParameterTypes.get(); - if (!someParameterIsUnknown(parameterTypes)) { - // give up and return all remaining matches - return mostSpecificFunctions; - } - - // look for functions that only cast the unknown arguments - List unknownOnlyCastFunctions = getUnknownOnlyCastFunctions(applicableFunctions, parameterTypes); - if (!unknownOnlyCastFunctions.isEmpty()) { - mostSpecificFunctions = unknownOnlyCastFunctions; + if (mostSpecificFunctions.size() > 1 && someParameterIsUnknown(parameters)) { + // if there is more than one function, look for functions that only cast the unknown arguments + List unknownOnlyCastFunctions = getUnknownOnlyCastFunctions(applicableFunctions, parameters); + if (!unknownOnlyCastFunctions.isEmpty()) { + mostSpecificFunctions = unknownOnlyCastFunctions; + } if (mostSpecificFunctions.size() == 1) { return mostSpecificFunctions; } + // If the return type for all the selected function is the same, and the parameters are not declared as nullable + // all the functions are semantically the same. We can return just any of those. + if (returnTypeIsTheSame(mostSpecificFunctions) && allReturnNullOnGivenInputTypes(mostSpecificFunctions, parameters)) { + // make it deterministic + ApplicableFunction selectedFunction = Ordering.usingToString() + .reverse() + .sortedCopy(mostSpecificFunctions) + .get(0); + return ImmutableList.of(selectedFunction); + } } - - // If the return type for all the selected function is the same, and the parameters are not declared as nullable - // all the functions are semantically the same. We can return just any of those. - if (returnTypeIsTheSame(mostSpecificFunctions) && allReturnNullOnGivenInputTypes(mostSpecificFunctions, parameterTypes)) { - // make it deterministic - ApplicableFunction selectedFunction = Ordering.usingToString() - .reverse() - .sortedCopy(mostSpecificFunctions) - .get(0); - return ImmutableList.of(selectedFunction); - } - return mostSpecificFunctions; } @@ -766,7 +725,7 @@ private List selectMostSpecificFunctions(List parameters) + private boolean someParameterIsUnknown(List parameters) { return parameters.stream().anyMatch(type -> type.equals(UNKNOWN)); } @@ -883,21 +842,20 @@ private SpecializedFunctionKey doGetSpecializedFunctionKey(Signature signature) Iterable candidates = functions.get(QualifiedName.of(signature.getName())); // search for exact match Type returnType = typeManager.getType(signature.getReturnType()); - List argumentTypeSignatureProviders = fromTypeSignatures(signature.getArgumentTypes()); + List argumentTypes = resolveTypes(signature.getArgumentTypes(), typeManager); for (SqlFunction candidate : candidates) { Optional boundVariables = new SignatureBinder(typeManager, candidate.getSignature(), false) - .bindVariables(argumentTypeSignatureProviders, returnType); + .bindVariables(argumentTypes, returnType); if (boundVariables.isPresent()) { - return new SpecializedFunctionKey(candidate, boundVariables.get(), argumentTypeSignatureProviders.size()); + return new SpecializedFunctionKey(candidate, boundVariables.get(), argumentTypes.size()); } } // TODO: hack because there could be "type only" coercions (which aren't necessarily included as implicit casts), // so do a second pass allowing "type only" coercions - List argumentTypes = resolveTypes(signature.getArgumentTypes(), typeManager); for (SqlFunction candidate : candidates) { SignatureBinder binder = new SignatureBinder(typeManager, candidate.getSignature(), true); - Optional boundVariables = binder.bindVariables(argumentTypeSignatureProviders, returnType); + Optional boundVariables = binder.bindVariables(argumentTypes, returnType); if (!boundVariables.isPresent()) { continue; } @@ -985,7 +943,7 @@ public Signature resolveOperator(OperatorType operatorType, List throws OperatorNotFoundException { try { - return resolveFunction(QualifiedName.of(mangleOperatorName(operatorType)), fromTypes(argumentTypes)); + return resolveFunction(QualifiedName.of(mangleOperatorName(operatorType)), Lists.transform(argumentTypes, Type::getTypeSignature)); } catch (PrestoException e) { if (e.getErrorCode().getCode() == FUNCTION_NOT_FOUND.toErrorCode().getCode()) { @@ -1078,24 +1036,12 @@ public static OperatorType unmangleOperator(String mangledName) return OperatorType.valueOf(mangledName.substring(OPERATOR_PREFIX.length())); } - public static Optional> toTypes(List typeSignatureProviders, TypeManager typeManager) - { - ImmutableList.Builder resultBuilder = ImmutableList.builder(); - for (TypeSignatureProvider typeSignatureProvider : typeSignatureProviders) { - if (typeSignatureProvider.hasDependency()) { - return Optional.empty(); - } - resultBuilder.add(typeManager.getType(typeSignatureProvider.getTypeSignature())); - } - return Optional.of(resultBuilder.build()); - } - /** * One method is more specific than another if invocation handled by the first method could be passed on to the other one */ - private boolean isMoreSpecificThan(ApplicableFunction left, ApplicableFunction right) + public boolean isMoreSpecificThan(ApplicableFunction left, ApplicableFunction right) { - List resolvedTypes = fromTypeSignatures(left.getBoundSignature().getArgumentTypes()); + List resolvedTypes = resolveTypes(left.getBoundSignature().getArgumentTypes(), typeManager); Optional boundVariables = new SignatureBinder(typeManager, right.getDeclaredSignature(), true) .bindVariables(resolvedTypes); return boundVariables.isPresent(); diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/InMemoryNodeManager.java b/presto-main/src/main/java/com/facebook/presto/metadata/InMemoryNodeManager.java index 64808e39b736..1e8a513923de 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/InMemoryNodeManager.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/InMemoryNodeManager.java @@ -84,7 +84,7 @@ public Set getActiveConnectorNodes(ConnectorId connectorId) @Override public AllNodes getAllNodes() { - return new AllNodes(ImmutableSet.builder().add(localNode).addAll(remoteNodes.values()).build(), ImmutableSet.of(), ImmutableSet.of()); + return new AllNodes(ImmutableSet.builder().add(localNode).addAll(remoteNodes.values()).build(), ImmutableSet.of(), ImmutableSet.of()); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/PolymorphicScalarFunction.java b/presto-main/src/main/java/com/facebook/presto/metadata/PolymorphicScalarFunction.java index 2f9f17ee2a3f..eb13e0fe1cb4 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/PolymorphicScalarFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/PolymorphicScalarFunction.java @@ -142,17 +142,17 @@ private boolean matchesParameterAndReturnTypes(Method method, List resolve return method.getReturnType().equals(getNullAwareContainerType(returnType.getJavaType(), nullableResult)); } - private static boolean onlyFirstMatchedMethodHasPredicate(MethodsGroup matchingMethodsGroup, MethodsGroup methodsGroup) + private boolean onlyFirstMatchedMethodHasPredicate(MethodsGroup matchingMethodsGroup, MethodsGroup methodsGroup) { return matchingMethodsGroup.getPredicate().isPresent() && !methodsGroup.getPredicate().isPresent(); } - private static boolean predicateIsTrue(MethodsGroup methodsGroup, SpecializeContext context) + private boolean predicateIsTrue(MethodsGroup methodsGroup, SpecializeContext context) { return methodsGroup.getPredicate().map(predicate -> predicate.test(context)).orElse(true); } - private static List computeExtraParameters(MethodsGroup methodsGroup, SpecializeContext context) + private List computeExtraParameters(MethodsGroup methodsGroup, SpecializeContext context) { return methodsGroup.getExtraParametersFunction().map(function -> function.apply(context)).orElse(emptyList()); } diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/Signature.java b/presto-main/src/main/java/com/facebook/presto/metadata/Signature.java index 10194a4ca1f8..0da40d638d26 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/Signature.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/Signature.java @@ -71,7 +71,7 @@ public Signature(String name, FunctionKind kind, TypeSignature returnType, TypeS public Signature(String name, FunctionKind kind, TypeSignature returnType, List argumentTypes) { - this(name, kind, ImmutableList.of(), ImmutableList.of(), returnType, argumentTypes, false); + this(name, kind, ImmutableList.of(), ImmutableList.of(), returnType, argumentTypes, false); } public static Signature internalOperator(OperatorType operator, Type returnType, List argumentTypes) @@ -101,7 +101,7 @@ public static Signature internalScalarFunction(String name, TypeSignature return public static Signature internalScalarFunction(String name, TypeSignature returnType, List argumentTypes) { - return new Signature(name, SCALAR, ImmutableList.of(), ImmutableList.of(), returnType, argumentTypes, false); + return new Signature(name, SCALAR, ImmutableList.of(), ImmutableList.of(), returnType, argumentTypes, false); } public Signature withAlias(String name) diff --git a/presto-main/src/main/java/com/facebook/presto/metadata/SignatureBinder.java b/presto-main/src/main/java/com/facebook/presto/metadata/SignatureBinder.java index 06e102463416..709aeb3f784c 100644 --- a/presto-main/src/main/java/com/facebook/presto/metadata/SignatureBinder.java +++ b/presto-main/src/main/java/com/facebook/presto/metadata/SignatureBinder.java @@ -19,8 +19,6 @@ import com.facebook.presto.spi.type.TypeManager; import com.facebook.presto.spi.type.TypeSignature; import com.facebook.presto.spi.type.TypeSignatureParameter; -import com.facebook.presto.sql.analyzer.TypeSignatureProvider; -import com.facebook.presto.type.FunctionType; import com.google.common.base.VerifyException; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -34,11 +32,9 @@ import java.util.Optional; import java.util.Set; -import static com.facebook.presto.sql.analyzer.TypeSignatureProvider.fromTypes; import static com.facebook.presto.type.TypeCalculation.calculateLiteralValue; import static com.facebook.presto.type.TypeRegistry.isCovariantTypeBase; import static com.facebook.presto.type.UnknownType.UNKNOWN; -import static com.facebook.presto.util.ImmutableCollectors.toImmutableList; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import static com.google.common.base.Verify.verify; @@ -86,7 +82,7 @@ public SignatureBinder(TypeManager typeManager, Signature declaredSignature, boo .collect(toMap(TypeVariableConstraint::getName, identity())); } - public Optional bind(List actualArgumentTypes) + public Optional bind(List actualArgumentTypes) { Optional boundVariables = bindVariables(actualArgumentTypes); if (!boundVariables.isPresent()) { @@ -95,7 +91,7 @@ public Optional bind(List actualArgu return Optional.of(applyBoundVariables(declaredSignature, boundVariables.get(), actualArgumentTypes.size())); } - public Optional bind(List actualArgumentTypes, Type actualReturnType) + public Optional bind(List actualArgumentTypes, Type actualReturnType) { Optional boundVariables = bindVariables(actualArgumentTypes, actualReturnType); if (!boundVariables.isPresent()) { @@ -104,7 +100,7 @@ public Optional bind(List actualArgu return Optional.of(applyBoundVariables(declaredSignature, boundVariables.get(), actualArgumentTypes.size())); } - public Optional bindVariables(List actualArgumentTypes) + public Optional bindVariables(List actualArgumentTypes) { ImmutableList.Builder constraintSolvers = ImmutableList.builder(); if (!appendConstraintSolversForArguments(constraintSolvers, actualArgumentTypes)) { @@ -114,10 +110,10 @@ public Optional bindVariables(List bindVariables(List actualArgumentTypes, Type actualReturnType) + public Optional bindVariables(List actualArgumentTypes, Type actualReturnType) { ImmutableList.Builder constraintSolvers = ImmutableList.builder(); - if (!appendConstraintSolversForReturnValue(constraintSolvers, new TypeSignatureProvider(actualReturnType.getTypeSignature()))) { + if (!appendConstraintSolversForReturnValue(constraintSolvers, actualReturnType)) { return Optional.empty(); } if (!appendConstraintSolversForArguments(constraintSolvers, actualArgumentTypes)) { @@ -211,14 +207,14 @@ private static void checkNoLiteralVariableUsageAcrossTypes(TypeSignature typeSig } } - private boolean appendConstraintSolversForReturnValue(ImmutableList.Builder resultBuilder, TypeSignatureProvider actualReturnType) + private boolean appendConstraintSolversForReturnValue(ImmutableList.Builder resultBuilder, Type actualReturnType) { TypeSignature formalReturnTypeSignature = declaredSignature.getReturnType(); - return appendTypeRelationshipConstraintSolver(resultBuilder, formalReturnTypeSignature, actualReturnType, false) - && appendConstraintSolvers(resultBuilder, formalReturnTypeSignature, actualReturnType, false); + appendTypeRelationshipConstraintSolver(resultBuilder, formalReturnTypeSignature, actualReturnType, false); + return appendConstraintSolvers(resultBuilder, formalReturnTypeSignature, actualReturnType, false); } - private boolean appendConstraintSolversForArguments(ImmutableList.Builder resultBuilder, List actualTypes) + private boolean appendConstraintSolversForArguments(ImmutableList.Builder resultBuilder, List actualTypes) { boolean variableArity = declaredSignature.isVariableArity(); List formalTypeSignatures = declaredSignature.getArgumentTypes(); @@ -234,9 +230,7 @@ private boolean appendConstraintSolversForArguments(ImmutableList.Builder resultBuilder, List formalTypeSignatures, - List actualTypes, + List actualTypes, boolean allowCoercion) { if (formalTypeSignatures.size() != actualTypes.size()) { @@ -262,35 +256,20 @@ private boolean appendConstraintSolvers( private boolean appendConstraintSolvers( ImmutableList.Builder resultBuilder, TypeSignature formalTypeSignature, - TypeSignatureProvider actualTypeSignatureProvider, + Type actualType, boolean allowCoercion) { - // formalTypeSignature can be categorized into one of the 5 cases below: - // * function type + // formalTypeSignature can be categorized into one of the 4 cases below: // * type without type parameter // * type parameter of type/named_type kind // * type with type parameter of literal/variable kind - // * type with type parameter of type/named_type kind (except function type) - - if (FunctionType.NAME.equals(formalTypeSignature.getBase())) { - List formalTypeParameterTypeSignatures = formalTypeSignature.getTypeParametersAsTypeSignatures(); - resultBuilder.add(new FunctionSolver( - getLambdaArgumentTypeSignatures(formalTypeSignature), - formalTypeParameterTypeSignatures.get(formalTypeParameterTypeSignatures.size() - 1), - actualTypeSignatureProvider)); - return true; - } - - if (actualTypeSignatureProvider.hasDependency()) { - return false; - } + // * type with type parameter of type/named_type kind if (formalTypeSignature.getParameters().isEmpty()) { TypeVariableConstraint typeVariableConstraint = typeVariableConstraints.get(formalTypeSignature.getBase()); if (typeVariableConstraint == null) { return true; } - Type actualType = typeManager.getType(actualTypeSignatureProvider.getTypeSignature()); resultBuilder.add(new TypeParameterSolver( formalTypeSignature.getBase(), actualType, @@ -300,23 +279,22 @@ private boolean appendConstraintSolvers( return true; } - Type actualType = typeManager.getType(actualTypeSignatureProvider.getTypeSignature()); if (isTypeWithLiteralParameters(formalTypeSignature)) { resultBuilder.add(new TypeWithLiteralParametersSolver(formalTypeSignature, actualType)); return true; } - List actualTypeParametersTypeSignatureProvider; + List actualTypeTypeParameters; if (UNKNOWN.equals(actualType)) { - actualTypeParametersTypeSignatureProvider = Collections.nCopies(formalTypeSignature.getParameters().size(), new TypeSignatureProvider(UNKNOWN.getTypeSignature())); + actualTypeTypeParameters = Collections.nCopies(formalTypeSignature.getParameters().size(), UNKNOWN); } else { - actualTypeParametersTypeSignatureProvider = fromTypes(actualType.getTypeParameters()); + actualTypeTypeParameters = actualType.getTypeParameters(); } ImmutableList.Builder formalTypeParameterTypeSignatures = ImmutableList.builder(); - for (TypeSignatureParameter formalTypeParameter : formalTypeSignature.getParameters()) { - Optional typeSignature = formalTypeParameter.getTypeSignatureOrNamedTypeSignature(); + for (TypeSignatureParameter formalTypeSignatureParameter : formalTypeSignature.getParameters()) { + Optional typeSignature = formalTypeSignatureParameter.getTypeSignatureOrNamedTypeSignature(); if (!typeSignature.isPresent()) { throw new UnsupportedOperationException("Types with both type parameters and literal parameters at the same time are not supported"); } @@ -326,7 +304,7 @@ private boolean appendConstraintSolvers( return appendConstraintSolvers( resultBuilder, formalTypeParameterTypeSignatures.build(), - actualTypeParametersTypeSignatureProvider, + actualTypeTypeParameters, allowCoercion && isCovariantTypeBase(formalTypeSignature.getBase())); } @@ -495,22 +473,6 @@ private static List expandVarargFormalTypeSignature(List getLambdaArgumentTypeSignatures(TypeSignature lambdaTypeSignature) - { - List typeParameters = lambdaTypeSignature.getTypeParametersAsTypeSignatures(); - return typeParameters.subList(0, typeParameters.size() - 1); - } - private interface TypeConstraintSolver { SolverReturnStatus update(BoundVariables.Builder bindings); @@ -690,111 +652,15 @@ public SolverReturnStatus update(BoundVariables.Builder bindings) } } - private class FunctionSolver - implements TypeConstraintSolver - { - private final List formalLambdaArgumentsTypeSignature; - private final TypeSignature formalLambdaReturnTypeSignature; - private final TypeSignatureProvider typeSignatureProvider; - - public FunctionSolver( - List formalLambdaArgumentsTypeSignature, - TypeSignature formalLambdaReturnTypeSignature, - TypeSignatureProvider typeSignatureProvider) - { - this.formalLambdaArgumentsTypeSignature = formalLambdaArgumentsTypeSignature; - this.formalLambdaReturnTypeSignature = formalLambdaReturnTypeSignature; - this.typeSignatureProvider = typeSignatureProvider; - } - - @Override - public SolverReturnStatus update(BoundVariables.Builder bindings) - { - Optional> lambdaArgumentTypes = synthesizeLambdaArgumentTypes(bindings, formalLambdaArgumentsTypeSignature); - if (!lambdaArgumentTypes.isPresent()) { - return SolverReturnStatus.UNCHANGED_NOT_SATISFIED; - } - TypeSignature actualLambdaTypeSignature; - if (!typeSignatureProvider.hasDependency()) { - actualLambdaTypeSignature = typeSignatureProvider.getTypeSignature(); - if (!getLambdaArgumentTypeSignatures(actualLambdaTypeSignature).equals(toTypeSignatures(lambdaArgumentTypes.get()))) { - return SolverReturnStatus.UNSOLVABLE; - } - } - else { - actualLambdaTypeSignature = typeSignatureProvider.getTypeSignature(lambdaArgumentTypes.get()); - verify(getLambdaArgumentTypeSignatures(actualLambdaTypeSignature).equals(toTypeSignatures(lambdaArgumentTypes.get()))); - } - - Type actualLambdaType = typeManager.getType(actualLambdaTypeSignature); - Type actualReturnType = ((FunctionType) actualLambdaType).getReturnType(); - - ImmutableList.Builder constraintsBuilder = ImmutableList.builder(); - // Coercion on function type is not supported yet. - if (!appendTypeRelationshipConstraintSolver(constraintsBuilder, formalLambdaReturnTypeSignature, new TypeSignatureProvider(actualReturnType.getTypeSignature()), false)) { - return SolverReturnStatus.UNSOLVABLE; - } - if (!appendConstraintSolvers(constraintsBuilder, formalLambdaReturnTypeSignature, new TypeSignatureProvider(actualReturnType.getTypeSignature()), false)) { - return SolverReturnStatus.UNSOLVABLE; - } - SolverReturnStatusMerger statusMerger = new SolverReturnStatusMerger(); - for (TypeConstraintSolver constraint : constraintsBuilder.build()) { - statusMerger.add(constraint.update(bindings)); - if (statusMerger.getCurrent() == SolverReturnStatus.UNSOLVABLE) { - return SolverReturnStatus.UNSOLVABLE; - } - } - return statusMerger.getCurrent(); - } - - private Optional> synthesizeLambdaArgumentTypes( - BoundVariables.Builder bindings, - List formalLambdaArgumentTypeSignatures) - { - ImmutableList.Builder lambdaArgumentTypesBuilder = ImmutableList.builder(); - for (TypeSignature lambdaArgument : formalLambdaArgumentTypeSignatures) { - if (typeVariableConstraints.containsKey(lambdaArgument.getBase())) { - if (!bindings.containsTypeVariable(lambdaArgument.getBase())) { - return Optional.empty(); - } - Type typeVariable = bindings.getTypeVariable(lambdaArgument.getBase()); - lambdaArgumentTypesBuilder.add(typeVariable); - } - else { - lambdaArgumentTypesBuilder.add(typeManager.getType(lambdaArgument)); - } - } - return Optional.of(lambdaArgumentTypesBuilder.build()); - } - - private List toTypeSignatures(List types) - { - return types.stream() - .map(Type::getTypeSignature) - .collect(toImmutableList()); - } - } - - private boolean appendTypeRelationshipConstraintSolver( + private void appendTypeRelationshipConstraintSolver( ImmutableList.Builder resultBuilder, TypeSignature formalTypeSignature, - TypeSignatureProvider actualTypeSignatureProvider, + Type actualType, boolean allowCoercion) { - if (actualTypeSignatureProvider.hasDependency()) { - // Fail if the formal type is not function. - // Otherwise do nothing because FunctionConstraintSolver will handle type relationship constraint directly - return FunctionType.NAME.equals(formalTypeSignature.getBase()); - } Set typeVariables = typeVariablesOf(formalTypeSignature); Set longVariables = longVariablesOf(formalTypeSignature); - resultBuilder.add(new TypeRelationshipConstraintSolver( - formalTypeSignature, - typeVariables, - longVariables, - typeManager.getType(actualTypeSignatureProvider.getTypeSignature()), - allowCoercion)); - return true; + resultBuilder.add(new TypeRelationshipConstraintSolver(formalTypeSignature, typeVariables, longVariables, actualType, allowCoercion)); } private class TypeRelationshipConstraintSolver @@ -831,7 +697,12 @@ public SolverReturnStatus update(BoundVariables.Builder bindings) TypeSignature boundSignature = applyBoundVariables(superTypeSignature, bindings.build()); - return satisfiesCoercion(allowCoercion, actualType, boundSignature) ? SolverReturnStatus.UNCHANGED_SATISFIED : SolverReturnStatus.UNSOLVABLE; + if (allowCoercion) { + return typeManager.canCoerce(actualType, typeManager.getType(boundSignature)) ? SolverReturnStatus.UNCHANGED_SATISFIED : SolverReturnStatus.UNSOLVABLE; + } + else { + return actualType.getTypeSignature().equals(boundSignature) ? SolverReturnStatus.UNCHANGED_SATISFIED : SolverReturnStatus.UNSOLVABLE; + } } } } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/DeleteOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/DeleteOperator.java index cf5fc7a26216..7e4f12330a71 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/DeleteOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/DeleteOperator.java @@ -39,7 +39,7 @@ public class DeleteOperator implements Operator { - public static final List TYPES = ImmutableList.of(BIGINT, VARBINARY); + public static final List TYPES = ImmutableList.of(BIGINT, VARBINARY); public static class DeleteOperatorFactory implements OperatorFactory diff --git a/presto-main/src/main/java/com/facebook/presto/operator/HashAggregationOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/HashAggregationOperator.java index 8db409be36cb..680a3475331c 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/HashAggregationOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/HashAggregationOperator.java @@ -17,18 +17,13 @@ import com.facebook.presto.operator.aggregation.AccumulatorFactory; import com.facebook.presto.operator.aggregation.builder.HashAggregationBuilder; import com.facebook.presto.operator.aggregation.builder.InMemoryHashAggregationBuilder; -import com.facebook.presto.operator.aggregation.builder.SpillableHashAggregationBuilder; import com.facebook.presto.spi.Page; import com.facebook.presto.spi.PageBuilder; import com.facebook.presto.spi.type.Type; -import com.facebook.presto.spiller.Spiller; -import com.facebook.presto.spiller.SpillerFactory; import com.facebook.presto.sql.planner.plan.AggregationNode.Step; import com.facebook.presto.sql.planner.plan.PlanNodeId; import com.facebook.presto.type.TypeUtils; -import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; -import com.google.common.util.concurrent.ListenableFuture; import io.airlift.units.DataSize; import java.util.Iterator; @@ -38,15 +33,11 @@ import static com.facebook.presto.operator.aggregation.builder.InMemoryHashAggregationBuilder.toTypes; import static com.google.common.base.Preconditions.checkState; -import static io.airlift.concurrent.MoreFutures.toListenableFuture; -import static io.airlift.units.DataSize.Unit.MEGABYTE; import static java.util.Objects.requireNonNull; public class HashAggregationOperator implements Operator { - private static final double MERGE_WITH_MEMORY_RATIO = 0.9; - public static class HashAggregationOperatorFactory implements OperatorFactory { @@ -63,10 +54,6 @@ public static class HashAggregationOperatorFactory private final int expectedGroups; private final List types; private final DataSize maxPartialMemory; - private final boolean spillEnabled; - private final DataSize memoryLimitBeforeSpill; - private final DataSize memoryLimitForMergeWithMemory; - private final SpillerFactory spillerFactory; private boolean closed; @@ -82,86 +69,6 @@ public HashAggregationOperatorFactory( Optional groupIdChannel, int expectedGroups, DataSize maxPartialMemory) - { - this(operatorId, - planNodeId, - groupByTypes, - groupByChannels, - globalAggregationGroupIds, - step, - accumulatorFactories, - hashChannel, - groupIdChannel, - expectedGroups, - maxPartialMemory, - false, - new DataSize(0, MEGABYTE), - new DataSize(0, MEGABYTE), - new SpillerFactory() { - @Override - public Spiller create(List types) - { - throw new UnsupportedOperationException(); - } - - @Override - public long getSpilledBytes() - { - return 0; - } - }); - } - - public HashAggregationOperatorFactory( - int operatorId, - PlanNodeId planNodeId, - List groupByTypes, - List groupByChannels, - List globalAggregationGroupIds, - Step step, - List accumulatorFactories, - Optional hashChannel, - Optional groupIdChannel, - int expectedGroups, - DataSize maxPartialMemory, - boolean spillEnabled, - DataSize memoryLimitBeforeSpill, - SpillerFactory spillerFactory) - { - this(operatorId, - planNodeId, - groupByTypes, - groupByChannels, - globalAggregationGroupIds, - step, - accumulatorFactories, - hashChannel, - groupIdChannel, - expectedGroups, - maxPartialMemory, - spillEnabled, - memoryLimitBeforeSpill, - DataSize.succinctBytes((long) (memoryLimitBeforeSpill.toBytes() * MERGE_WITH_MEMORY_RATIO)), - spillerFactory); - } - - @VisibleForTesting - HashAggregationOperatorFactory( - int operatorId, - PlanNodeId planNodeId, - List groupByTypes, - List groupByChannels, - List globalAggregationGroupIds, - Step step, - List accumulatorFactories, - Optional hashChannel, - Optional groupIdChannel, - int expectedGroups, - DataSize maxPartialMemory, - boolean spillEnabled, - DataSize memoryLimitBeforeSpill, - DataSize memoryLimitForMergeWithMemory, - SpillerFactory spillerFactory) { this.operatorId = operatorId; this.planNodeId = requireNonNull(planNodeId, "planNodeId is null"); @@ -174,10 +81,6 @@ public HashAggregationOperatorFactory( this.accumulatorFactories = ImmutableList.copyOf(accumulatorFactories); this.expectedGroups = expectedGroups; this.maxPartialMemory = requireNonNull(maxPartialMemory, "maxPartialMemory is null"); - this.spillEnabled = spillEnabled; - this.memoryLimitBeforeSpill = requireNonNull(memoryLimitBeforeSpill, "memoryLimitBeforeSpill is null"); - this.memoryLimitForMergeWithMemory = requireNonNull(memoryLimitForMergeWithMemory, "memoryLimitForMergeWithMemory is null"); - this.spillerFactory = requireNonNull(spillerFactory, "spillerFactory is null"); this.types = toTypes(groupByTypes, step, accumulatorFactories, hashChannel); } @@ -204,11 +107,7 @@ public Operator createOperator(DriverContext driverContext) hashChannel, groupIdChannel, expectedGroups, - maxPartialMemory, - spillEnabled, - memoryLimitBeforeSpill, - memoryLimitForMergeWithMemory, - spillerFactory); + maxPartialMemory); return hashAggregationOperator; } @@ -232,11 +131,7 @@ public OperatorFactory duplicate() hashChannel, groupIdChannel, expectedGroups, - maxPartialMemory, - spillEnabled, - memoryLimitBeforeSpill, - memoryLimitForMergeWithMemory, - spillerFactory); + maxPartialMemory); } } @@ -250,10 +145,6 @@ public OperatorFactory duplicate() private final Optional groupIdChannel; private final int expectedGroups; private final DataSize maxPartialMemory; - private final boolean spillEnabled; - private final DataSize memoryLimitBeforeSpill; - private final DataSize memoryLimitForMergeWithMemory; - private final SpillerFactory spillerFactory; private final List types; @@ -273,11 +164,7 @@ public HashAggregationOperator( Optional hashChannel, Optional groupIdChannel, int expectedGroups, - DataSize maxPartialMemory, - boolean spillEnabled, - DataSize memoryLimitBeforeSpill, - DataSize memoryLimitForMergeWithMemory, - SpillerFactory spillerFactory) + DataSize maxPartialMemory) { this.operatorContext = requireNonNull(operatorContext, "operatorContext is null"); requireNonNull(step, "step is null"); @@ -294,10 +181,6 @@ public HashAggregationOperator( this.expectedGroups = expectedGroups; this.maxPartialMemory = requireNonNull(maxPartialMemory, "maxPartialMemory is null"); this.types = toTypes(groupByTypes, step, accumulatorFactories, hashChannel); - this.spillEnabled = spillEnabled; - this.memoryLimitBeforeSpill = requireNonNull(memoryLimitBeforeSpill, "memoryLimitBeforeSpill is null"); - this.memoryLimitForMergeWithMemory = requireNonNull(memoryLimitForMergeWithMemory, "memoryLimitForMergeWithMemory is null"); - this.spillerFactory = requireNonNull(spillerFactory, "spillerFactory is null"); } @Override @@ -324,27 +207,10 @@ public boolean isFinished() return finished; } - @Override - public ListenableFuture isBlocked() - { - if (aggregationBuilder != null) { - return toListenableFuture(aggregationBuilder.isBlocked()); - } - return NOT_BLOCKED; - } - @Override public boolean needsInput() { - if (finishing || outputIterator != null) { - return false; - } - else if (aggregationBuilder != null && aggregationBuilder.isFull()) { - return false; - } - else { - return true; - } + return !finishing && outputIterator == null && (aggregationBuilder == null || !aggregationBuilder.isFull()); } @Override @@ -355,30 +221,15 @@ public void addInput(Page page) inputProcessed = true; if (aggregationBuilder == null) { - if (step.isOutputPartial() || !spillEnabled) { - aggregationBuilder = new InMemoryHashAggregationBuilder( - accumulatorFactories, - step, - expectedGroups, - groupByTypes, - groupByChannels, - hashChannel, - operatorContext, - maxPartialMemory); - } - else { - aggregationBuilder = new SpillableHashAggregationBuilder( - accumulatorFactories, - step, - expectedGroups, - groupByTypes, - groupByChannels, - hashChannel, - operatorContext, - memoryLimitBeforeSpill, - memoryLimitForMergeWithMemory, - spillerFactory); - } + aggregationBuilder = new InMemoryHashAggregationBuilder( + accumulatorFactories, + step, + expectedGroups, + groupByTypes, + groupByChannels, + hashChannel, + operatorContext, + maxPartialMemory); // assume initial aggregationBuilder is not full } @@ -386,7 +237,6 @@ public void addInput(Page page) checkState(!aggregationBuilder.isFull(), "Aggregation buffer is full"); } aggregationBuilder.processPage(page); - aggregationBuilder.updateMemory(); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/operator/HttpPageBufferClient.java b/presto-main/src/main/java/com/facebook/presto/operator/HttpPageBufferClient.java index 04255ee245ce..c206f9d3fbdf 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/HttpPageBufferClient.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/HttpPageBufferClient.java @@ -654,7 +654,7 @@ public static PagesResponse createPagesResponse(String taskInstanceId, long toke public static PagesResponse createEmptyPagesResponse(String taskInstanceId, long token, long nextToken, boolean complete) { - return new PagesResponse(taskInstanceId, token, nextToken, ImmutableList.of(), complete); + return new PagesResponse(taskInstanceId, token, nextToken, ImmutableList.of(), complete); } private final String taskInstanceId; diff --git a/presto-main/src/main/java/com/facebook/presto/operator/MergeHashSort.java b/presto-main/src/main/java/com/facebook/presto/operator/MergeHashSort.java index abea1aaa8b8f..a6568524a6ec 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/MergeHashSort.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/MergeHashSort.java @@ -38,9 +38,6 @@ private MergeHashSort() { } - /** - * Rows with same hash value are guaranteed to be in the same result page. - */ public static Iterator merge(List keyTypes, List allTypes, List> channels) { List> channelIterators = channels.stream().map(SingleChannelPagePositions::new).collect(toList()); @@ -104,7 +101,7 @@ public boolean isPositionOutOfPage() } } - public interface PagePositions extends Iterator + public static interface PagePositions extends Iterator { } @@ -140,6 +137,7 @@ public PagePosition next() /** * This class rewrites iterator over PagePosition to iterator over Pages. + * Positions with same hash value are guaranteed to be in the same result page. */ public static class PageRewriteIterator implements Iterator diff --git a/presto-main/src/main/java/com/facebook/presto/operator/MetadataDeleteOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/MetadataDeleteOperator.java index a70004a17ef4..5a1487ed512b 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/MetadataDeleteOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/MetadataDeleteOperator.java @@ -34,7 +34,7 @@ public class MetadataDeleteOperator implements Operator { - public static final List TYPES = ImmutableList.of(BIGINT); + public static final List TYPES = ImmutableList.of(BIGINT); public static class MetadataDeleteOperatorFactory implements OperatorFactory diff --git a/presto-main/src/main/java/com/facebook/presto/operator/NestedLoopJoinOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/NestedLoopJoinOperator.java index dbf1fcff686a..e46cbff28b75 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/NestedLoopJoinOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/NestedLoopJoinOperator.java @@ -272,13 +272,11 @@ private static int calculateUseNoColumnShortcut( return -1; } - @Override public boolean hasNext() { return rowIndex < maxRowIndex; } - @Override public Page next() { if (!hasNext()) { diff --git a/presto-main/src/main/java/com/facebook/presto/operator/PageSourceOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/PageSourceOperator.java index ac6d838ad44b..c25921e03cd9 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/PageSourceOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/PageSourceOperator.java @@ -18,14 +18,11 @@ import com.facebook.presto.spi.type.Type; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; -import com.google.common.util.concurrent.ListenableFuture; import java.io.Closeable; import java.io.IOException; import java.util.List; -import java.util.concurrent.CompletableFuture; -import static io.airlift.concurrent.MoreFutures.toListenableFuture; import static java.util.Objects.requireNonNull; public class PageSourceOperator @@ -73,13 +70,6 @@ public boolean isFinished() return pageSource.isFinished(); } - @Override - public ListenableFuture isBlocked() - { - CompletableFuture pageSourceBlocked = pageSource.isBlocked(); - return pageSourceBlocked.isDone() ? NOT_BLOCKED : toListenableFuture(pageSourceBlocked); - } - @Override public boolean needsInput() { diff --git a/presto-main/src/main/java/com/facebook/presto/operator/PagesIndex.java b/presto-main/src/main/java/com/facebook/presto/operator/PagesIndex.java index 39dfd9a60eb0..d739883bbbb8 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/PagesIndex.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/PagesIndex.java @@ -321,7 +321,7 @@ public PagesHashStrategy createPagesHashStrategy(List joinChannels, Opt } // if compilation fails, use interpreter - return new SimplePagesHashStrategy(types, ImmutableList.copyOf(channels), joinChannels, hashChannel); + return new SimplePagesHashStrategy(types, ImmutableList.>copyOf(channels), joinChannels, hashChannel); } public Supplier createLookupSourceSupplier( diff --git a/presto-main/src/main/java/com/facebook/presto/operator/PagesIndexOrdering.java b/presto-main/src/main/java/com/facebook/presto/operator/PagesIndexOrdering.java index bc0f94ac5fac..b6c49b9faaaa 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/PagesIndexOrdering.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/PagesIndexOrdering.java @@ -154,7 +154,7 @@ private int median3(PagesIndex pagesIndex, int a, int b, int c) /** * Swaps x[a .. (a+n-1)] with x[b .. (b+n-1)]. */ - private static void vectorSwap(PagesIndex pagesIndex, int from, int l, int s) + private void vectorSwap(PagesIndex pagesIndex, int from, int l, int s) { for (int i = 0; i < s; i++, from++, l++) { pagesIndex.swap(from, l); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/PipelineStats.java b/presto-main/src/main/java/com/facebook/presto/operator/PipelineStats.java index 9ba12da3972e..768ddd5aac1f 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/PipelineStats.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/PipelineStats.java @@ -371,6 +371,6 @@ public PipelineStats summarize() outputDataSize, outputPositions, operatorSummaries, - ImmutableList.of()); + ImmutableList.of()); } } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/ScanFilterAndProjectOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/ScanFilterAndProjectOperator.java index 529a79c5ac24..84536cdea719 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/ScanFilterAndProjectOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/ScanFilterAndProjectOperator.java @@ -34,7 +34,6 @@ import java.io.IOException; import java.util.List; import java.util.Optional; -import java.util.concurrent.CompletableFuture; import java.util.function.Supplier; import static com.facebook.presto.SystemSessionProperties.getProcessingOptimization; @@ -42,7 +41,6 @@ import static com.facebook.presto.sql.analyzer.FeaturesConfig.ProcessingOptimization.COLUMNAR_DICTIONARY; import static com.facebook.presto.sql.analyzer.FeaturesConfig.ProcessingOptimization.DISABLED; import static com.google.common.base.Preconditions.checkState; -import static io.airlift.concurrent.MoreFutures.toListenableFuture; import static java.util.Objects.requireNonNull; public class ScanFilterAndProjectOperator @@ -188,14 +186,7 @@ public final boolean isFinished() @Override public ListenableFuture isBlocked() { - if (!blocked.isDone()) { - return blocked; - } - if (pageSource != null) { - CompletableFuture pageSourceBlocked = pageSource.isBlocked(); - return pageSourceBlocked.isDone() ? NOT_BLOCKED : toListenableFuture(pageSourceBlocked); - } - return NOT_BLOCKED; + return blocked; } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/operator/TableFinishOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/TableFinishOperator.java index 8cb9734b3735..dd234abb590e 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/TableFinishOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/TableFinishOperator.java @@ -32,7 +32,7 @@ public class TableFinishOperator implements Operator { - public static final List TYPES = ImmutableList.of(BIGINT); + public static final List TYPES = ImmutableList.of(BIGINT); public static class TableFinishOperatorFactory implements OperatorFactory diff --git a/presto-main/src/main/java/com/facebook/presto/operator/TableScanOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/TableScanOperator.java index 593a316edc33..3eba472068a3 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/TableScanOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/TableScanOperator.java @@ -31,11 +31,9 @@ import java.io.IOException; import java.util.List; import java.util.Optional; -import java.util.concurrent.CompletableFuture; import java.util.function.Supplier; import static com.google.common.base.Preconditions.checkState; -import static io.airlift.concurrent.MoreFutures.toListenableFuture; import static java.util.Objects.requireNonNull; public class TableScanOperator @@ -221,14 +219,7 @@ public boolean isFinished() @Override public ListenableFuture isBlocked() { - if (!blocked.isDone()) { - return blocked; - } - if (source != null) { - CompletableFuture pageSourceBlocked = source.isBlocked(); - return pageSourceBlocked.isDone() ? NOT_BLOCKED : toListenableFuture(pageSourceBlocked); - } - return NOT_BLOCKED; + return blocked; } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/operator/TableWriterOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/TableWriterOperator.java index 224707e2d11d..81fc7f24335c 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/TableWriterOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/TableWriterOperator.java @@ -44,7 +44,7 @@ public class TableWriterOperator implements Operator { - public static final List TYPES = ImmutableList.of(BIGINT, VARBINARY); + public static final List TYPES = ImmutableList.of(BIGINT, VARBINARY); public static class TableWriterOperatorFactory implements OperatorFactory diff --git a/presto-main/src/main/java/com/facebook/presto/operator/TaskStats.java b/presto-main/src/main/java/com/facebook/presto/operator/TaskStats.java index a1fd28f82a9b..15373ff37ad9 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/TaskStats.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/TaskStats.java @@ -101,7 +101,7 @@ public TaskStats(DateTime createTime, DateTime endTime) 0, new DataSize(0, BYTE), 0, - ImmutableList.of()); + ImmutableList.of()); } @JsonCreator @@ -401,6 +401,6 @@ public TaskStats summarize() processedInputPositions, outputDataSize, outputPositions, - ImmutableList.of()); + ImmutableList.of()); } } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/WindowOperator.java b/presto-main/src/main/java/com/facebook/presto/operator/WindowOperator.java index 61f9dd6a78b1..de74409322b8 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/WindowOperator.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/WindowOperator.java @@ -224,7 +224,7 @@ public WindowOperator( this.pagesIndex = new PagesIndex(sourceTypes, expectedPositions); this.preGroupedChannels = Ints.toArray(preGroupedChannels); - this.preGroupedPartitionHashStrategy = pagesIndex.createPagesHashStrategy(preGroupedChannels, Optional.empty()); + this.preGroupedPartitionHashStrategy = pagesIndex.createPagesHashStrategy(preGroupedChannels, Optional.empty()); List unGroupedPartitionChannels = partitionChannels.stream() .filter(channel -> !preGroupedChannels.contains(channel)) .collect(toImmutableList()); @@ -232,7 +232,7 @@ public WindowOperator( List preSortedChannels = sortChannels.stream() .limit(preSortedChannelPrefix) .collect(toImmutableList()); - this.preSortedPartitionHashStrategy = pagesIndex.createPagesHashStrategy(preSortedChannels, Optional.empty()); + this.preSortedPartitionHashStrategy = pagesIndex.createPagesHashStrategy(preSortedChannels, Optional.empty()); this.peerGroupHashStrategy = pagesIndex.createPagesHashStrategy(sortChannels, Optional.empty()); this.pageBuilder = new PageBuilder(this.types); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxAggregationFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxAggregationFunction.java index 6fd366c91722..49fc430c4329 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxAggregationFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxAggregationFunction.java @@ -76,6 +76,8 @@ public abstract class AbstractMinMaxAggregationFunction private final OperatorType operatorType; + private final StateCompiler compiler = new StateCompiler(); + protected AbstractMinMaxAggregationFunction(String name, OperatorType operatorType) { super(name, @@ -109,28 +111,28 @@ protected InternalAggregationFunction generateAggregation(Type type, MethodHandl if (type.getJavaType() == long.class) { stateInterface = NullableLongState.class; - stateSerializer = StateCompiler.generateStateSerializer(stateInterface, classLoader); + stateSerializer = compiler.generateStateSerializer(stateInterface, classLoader); inputFunction = LONG_INPUT_FUNCTION; combineFunction = LONG_COMBINE_FUNCTION; outputFunction = LONG_OUTPUT_FUNCTION; } else if (type.getJavaType() == double.class) { stateInterface = NullableDoubleState.class; - stateSerializer = StateCompiler.generateStateSerializer(stateInterface, classLoader); + stateSerializer = compiler.generateStateSerializer(stateInterface, classLoader); inputFunction = DOUBLE_INPUT_FUNCTION; combineFunction = DOUBLE_COMBINE_FUNCTION; outputFunction = DOUBLE_OUTPUT_FUNCTION; } else if (type.getJavaType() == Slice.class) { stateInterface = SliceState.class; - stateSerializer = StateCompiler.generateStateSerializer(stateInterface, classLoader); + stateSerializer = compiler.generateStateSerializer(stateInterface, classLoader); inputFunction = SLICE_INPUT_FUNCTION; combineFunction = SLICE_COMBINE_FUNCTION; outputFunction = SLICE_OUTPUT_FUNCTION; } else if (type.getJavaType() == boolean.class) { stateInterface = NullableBooleanState.class; - stateSerializer = StateCompiler.generateStateSerializer(stateInterface, classLoader); + stateSerializer = compiler.generateStateSerializer(stateInterface, classLoader); inputFunction = BOOLEAN_INPUT_FUNCTION; combineFunction = BOOLEAN_COMBINE_FUNCTION; outputFunction = BOOLEAN_OUTPUT_FUNCTION; @@ -147,7 +149,7 @@ else if (type.getJavaType() == boolean.class) { combineFunction = combineFunction.bindTo(compareMethodHandle); outputFunction = outputFunction.bindTo(type); - AccumulatorStateFactory stateFactory = StateCompiler.generateStateFactory(stateInterface, classLoader); + AccumulatorStateFactory stateFactory = compiler.generateStateFactory(stateInterface, classLoader); Type intermediateType = stateSerializer.getSerializedType(); AggregationMetadata metadata = new AggregationMetadata( @@ -161,7 +163,7 @@ else if (type.getJavaType() == boolean.class) { stateFactory, type); - GenericAccumulatorFactoryBinder factory = AccumulatorCompiler.generateAccumulatorFactoryBinder(metadata, classLoader); + GenericAccumulatorFactoryBinder factory = new AccumulatorCompiler().generateAccumulatorFactoryBinder(metadata, classLoader); return new InternalAggregationFunction(getSignature().getName(), inputTypes, intermediateType, type, true, factory); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxBy.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxBy.java index a394645910b5..2d0ac6fc5144 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxBy.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxBy.java @@ -89,7 +89,7 @@ private InternalAggregationFunction generateAggregation(Type valueType, Type key stateFactory, valueType); - GenericAccumulatorFactoryBinder factory = AccumulatorCompiler.generateAccumulatorFactoryBinder(metadata, classLoader); + GenericAccumulatorFactoryBinder factory = new AccumulatorCompiler().generateAccumulatorFactoryBinder(metadata, classLoader); return new InternalAggregationFunction(getSignature().getName(), inputTypes, intermediateType, valueType, true, factory); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxByNAggregationFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxByNAggregationFunction.java index a8b33067a810..0409927b21bd 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxByNAggregationFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxByNAggregationFunction.java @@ -163,7 +163,7 @@ protected InternalAggregationFunction generateAggregation(Type valueType, Type k new MinMaxByNStateFactory(), outputType); - GenericAccumulatorFactoryBinder factory = AccumulatorCompiler.generateAccumulatorFactoryBinder(metadata, classLoader); + GenericAccumulatorFactoryBinder factory = new AccumulatorCompiler().generateAccumulatorFactoryBinder(metadata, classLoader); return new InternalAggregationFunction(name, inputTypes, intermediateType, outputType, true, factory); } } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxNAggregationFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxNAggregationFunction.java index 31d85610d34c..310530d1ad25 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxNAggregationFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AbstractMinMaxNAggregationFunction.java @@ -69,7 +69,6 @@ protected AbstractMinMaxNAggregationFunction(String name, Function accumulatorClass = generateAccumulatorClass( Accumulator.class, diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AggregationCompiler.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AggregationCompiler.java index 4875704dcbcd..efb9b3ab15f5 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AggregationCompiler.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/AggregationCompiler.java @@ -87,7 +87,7 @@ public static List generateBindableAggregationFunct ImmutableList.Builder builder = ImmutableList.builder(); for (Class stateClass : getStateClasses(aggregationDefinition)) { - AccumulatorStateSerializer stateSerializer = StateCompiler.generateStateSerializer(stateClass, classLoader); + AccumulatorStateSerializer stateSerializer = new StateCompiler().generateStateSerializer(stateClass, classLoader); for (Method outputFunction : getOutputFunctions(aggregationDefinition, stateClass)) { for (Method inputFunction : getInputFunctions(aggregationDefinition, stateClass)) { diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/ApproximateSetAggregation.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/ApproximateSetAggregation.java index cc1d15fa4c22..1df98086abfe 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/ApproximateSetAggregation.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/ApproximateSetAggregation.java @@ -31,7 +31,7 @@ public final class ApproximateSetAggregation { private static final int NUMBER_OF_BUCKETS = 4096; - private static final AccumulatorStateSerializer SERIALIZER = StateCompiler.generateStateSerializer(HyperLogLogState.class); + private static final AccumulatorStateSerializer SERIALIZER = new StateCompiler().generateStateSerializer(HyperLogLogState.class); private ApproximateSetAggregation() {} diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/ArbitraryAggregationFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/ArbitraryAggregationFunction.java index f52283ae3f80..67a0bea6d14c 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/ArbitraryAggregationFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/ArbitraryAggregationFunction.java @@ -70,6 +70,8 @@ public class ArbitraryAggregationFunction private static final MethodHandle BOOLEAN_COMBINE_FUNCTION = methodHandle(ArbitraryAggregationFunction.class, "combine", NullableBooleanState.class, NullableBooleanState.class); private static final MethodHandle BLOCK_COMBINE_FUNCTION = methodHandle(ArbitraryAggregationFunction.class, "combine", BlockState.class, BlockState.class); + private static final StateCompiler compiler = new StateCompiler(); + protected ArbitraryAggregationFunction() { super(NAME, @@ -106,28 +108,28 @@ private static InternalAggregationFunction generateAggregation(Type type) if (type.getJavaType() == long.class) { stateInterface = NullableLongState.class; - stateSerializer = StateCompiler.generateStateSerializer(stateInterface, classLoader); + stateSerializer = compiler.generateStateSerializer(stateInterface, classLoader); inputFunction = LONG_INPUT_FUNCTION; combineFunction = LONG_COMBINE_FUNCTION; outputFunction = LONG_OUTPUT_FUNCTION; } else if (type.getJavaType() == double.class) { stateInterface = NullableDoubleState.class; - stateSerializer = StateCompiler.generateStateSerializer(stateInterface, classLoader); + stateSerializer = compiler.generateStateSerializer(stateInterface, classLoader); inputFunction = DOUBLE_INPUT_FUNCTION; combineFunction = DOUBLE_COMBINE_FUNCTION; outputFunction = DOUBLE_OUTPUT_FUNCTION; } else if (type.getJavaType() == Slice.class) { stateInterface = SliceState.class; - stateSerializer = StateCompiler.generateStateSerializer(stateInterface, classLoader); + stateSerializer = compiler.generateStateSerializer(stateInterface, classLoader); inputFunction = SLICE_INPUT_FUNCTION; combineFunction = SLICE_COMBINE_FUNCTION; outputFunction = SLICE_OUTPUT_FUNCTION; } else if (type.getJavaType() == boolean.class) { stateInterface = NullableBooleanState.class; - stateSerializer = StateCompiler.generateStateSerializer(stateInterface, classLoader); + stateSerializer = compiler.generateStateSerializer(stateInterface, classLoader); inputFunction = BOOLEAN_INPUT_FUNCTION; combineFunction = BOOLEAN_COMBINE_FUNCTION; outputFunction = BOOLEAN_OUTPUT_FUNCTION; @@ -151,10 +153,10 @@ else if (type.getJavaType() == boolean.class) { outputFunction.bindTo(type), stateInterface, stateSerializer, - StateCompiler.generateStateFactory(stateInterface, classLoader), + compiler.generateStateFactory(stateInterface, classLoader), type); - GenericAccumulatorFactoryBinder factory = AccumulatorCompiler.generateAccumulatorFactoryBinder(metadata, classLoader); + GenericAccumulatorFactoryBinder factory = new AccumulatorCompiler().generateAccumulatorFactoryBinder(metadata, classLoader); return new InternalAggregationFunction(NAME, inputTypes, intermediateType, type, true, factory); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/ArrayAggregationFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/ArrayAggregationFunction.java index 0ceea16da2e4..afaea282e152 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/ArrayAggregationFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/ArrayAggregationFunction.java @@ -106,7 +106,7 @@ private static InternalAggregationFunction generateAggregation(Type type, boolea stateFactory, outputType); - GenericAccumulatorFactoryBinder factory = AccumulatorCompiler.generateAccumulatorFactoryBinder(metadata, classLoader); + GenericAccumulatorFactoryBinder factory = new AccumulatorCompiler().generateAccumulatorFactoryBinder(metadata, classLoader); return new InternalAggregationFunction(NAME, inputTypes, intermediateType, outputType, true, factory); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/BindableAggregationFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/BindableAggregationFunction.java index bf84ca1ab671..ed4aa1e17da0 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/BindableAggregationFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/BindableAggregationFunction.java @@ -98,10 +98,10 @@ public InternalAggregationFunction specialize(BoundVariables variables, int arit DynamicClassLoader classLoader = new DynamicClassLoader(definitionClass.getClassLoader(), getClass().getClassLoader()); AggregationMetadata metadata; - AccumulatorStateSerializer stateSerializer = StateCompiler.generateStateSerializer(stateClass, classLoader); + AccumulatorStateSerializer stateSerializer = new StateCompiler().generateStateSerializer(stateClass, classLoader); Type intermediateType = stateSerializer.getSerializedType(); Method combineFunction = AggregationCompiler.getCombineFunction(definitionClass, stateClass); - AccumulatorStateFactory stateFactory = StateCompiler.generateStateFactory(stateClass, classLoader); + AccumulatorStateFactory stateFactory = new StateCompiler().generateStateFactory(stateClass, classLoader); try { MethodHandle inputHandle = lookup().unreflect(inputFunction); diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/ChecksumAggregationFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/ChecksumAggregationFunction.java index a65665327818..8cc547329733 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/ChecksumAggregationFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/ChecksumAggregationFunction.java @@ -89,11 +89,11 @@ private static InternalAggregationFunction generateAggregation(Type type) COMBINE_FUNCTION, OUTPUT_FUNCTION, NullableLongState.class, - StateCompiler.generateStateSerializer(NullableLongState.class, classLoader), - StateCompiler.generateStateFactory(NullableLongState.class, classLoader), + new StateCompiler().generateStateSerializer(NullableLongState.class, classLoader), + new StateCompiler().generateStateFactory(NullableLongState.class, classLoader), VARBINARY); - GenericAccumulatorFactoryBinder factory = AccumulatorCompiler.generateAccumulatorFactoryBinder(metadata, classLoader); + GenericAccumulatorFactoryBinder factory = new AccumulatorCompiler().generateAccumulatorFactoryBinder(metadata, classLoader); return new InternalAggregationFunction(NAME, inputTypes, BIGINT, VARBINARY, true, factory); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/CountColumn.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/CountColumn.java index 4f05cfc50ea2..99e8a85753f7 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/CountColumn.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/CountColumn.java @@ -77,8 +77,8 @@ private static InternalAggregationFunction generateAggregation(Type type) { DynamicClassLoader classLoader = new DynamicClassLoader(CountColumn.class.getClassLoader()); - AccumulatorStateSerializer stateSerializer = StateCompiler.generateStateSerializer(LongState.class, classLoader); - AccumulatorStateFactory stateFactory = StateCompiler.generateStateFactory(LongState.class, classLoader); + AccumulatorStateSerializer stateSerializer = new StateCompiler().generateStateSerializer(LongState.class, classLoader); + AccumulatorStateFactory stateFactory = new StateCompiler().generateStateFactory(LongState.class, classLoader); Type intermediateType = stateSerializer.getSerializedType(); List inputTypes = ImmutableList.of(type); @@ -94,7 +94,7 @@ private static InternalAggregationFunction generateAggregation(Type type) stateFactory, BIGINT); - GenericAccumulatorFactoryBinder factory = AccumulatorCompiler.generateAccumulatorFactoryBinder(metadata, classLoader); + GenericAccumulatorFactoryBinder factory = new AccumulatorCompiler().generateAccumulatorFactoryBinder(metadata, classLoader); return new InternalAggregationFunction(NAME, inputTypes, intermediateType, BIGINT, true, factory); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/DecimalAverageAggregation.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/DecimalAverageAggregation.java index 38e8de688f00..2c5a37dbe320 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/DecimalAverageAggregation.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/DecimalAverageAggregation.java @@ -120,7 +120,7 @@ private static InternalAggregationFunction generateAggregation(Type type) type); Type intermediateType = stateSerializer.getSerializedType(); - GenericAccumulatorFactoryBinder factory = AccumulatorCompiler.generateAccumulatorFactoryBinder(metadata, classLoader); + GenericAccumulatorFactoryBinder factory = new AccumulatorCompiler().generateAccumulatorFactoryBinder(metadata, classLoader); return new InternalAggregationFunction(NAME, inputTypes, intermediateType, type, true, factory); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/DecimalSumAggregation.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/DecimalSumAggregation.java index db3377c128e1..4593f0c71900 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/DecimalSumAggregation.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/DecimalSumAggregation.java @@ -115,7 +115,7 @@ private static InternalAggregationFunction generateAggregation(Type inputType, T outputType); Type intermediateType = stateSerializer.getSerializedType(); - GenericAccumulatorFactoryBinder factory = AccumulatorCompiler.generateAccumulatorFactoryBinder(metadata, classLoader); + GenericAccumulatorFactoryBinder factory = new AccumulatorCompiler().generateAccumulatorFactoryBinder(metadata, classLoader); return new InternalAggregationFunction(NAME, inputTypes, intermediateType, outputType, true, factory); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/Histogram.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/Histogram.java index b774b34ae436..bef014ae584c 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/Histogram.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/Histogram.java @@ -100,7 +100,7 @@ private static InternalAggregationFunction generateAggregation(Type keyType, Typ new HistogramStateFactory(), outputType); - GenericAccumulatorFactoryBinder factory = AccumulatorCompiler.generateAccumulatorFactoryBinder(metadata, classLoader); + GenericAccumulatorFactoryBinder factory = new AccumulatorCompiler().generateAccumulatorFactoryBinder(metadata, classLoader); return new InternalAggregationFunction(NAME, inputTypes, intermediateType, outputType, true, factory); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/LazyAccumulatorFactoryBinder.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/LazyAccumulatorFactoryBinder.java index 7f54e5cc0c97..f654771641e0 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/LazyAccumulatorFactoryBinder.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/LazyAccumulatorFactoryBinder.java @@ -27,7 +27,7 @@ public class LazyAccumulatorFactoryBinder public LazyAccumulatorFactoryBinder(AggregationMetadata metadata, DynamicClassLoader classLoader) { - binder = Suppliers.memoize(() -> AccumulatorCompiler.generateAccumulatorFactoryBinder(metadata, classLoader)); + binder = Suppliers.memoize(() -> new AccumulatorCompiler().generateAccumulatorFactoryBinder(metadata, classLoader)); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/MapAggregationFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/MapAggregationFunction.java index c751c5ce770a..12d5b2e545bb 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/MapAggregationFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/MapAggregationFunction.java @@ -97,7 +97,7 @@ private static InternalAggregationFunction generateAggregation(Type keyType, Typ new KeyValuePairsStateFactory(keyType, valueType), outputType); - GenericAccumulatorFactoryBinder factory = AccumulatorCompiler.generateAccumulatorFactoryBinder(metadata, classLoader); + GenericAccumulatorFactoryBinder factory = new AccumulatorCompiler().generateAccumulatorFactoryBinder(metadata, classLoader); return new InternalAggregationFunction(NAME, inputTypes, intermediateType, outputType, true, factory); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/MapUnionAggregation.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/MapUnionAggregation.java index 35896e77937a..3d27954607b2 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/MapUnionAggregation.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/MapUnionAggregation.java @@ -87,7 +87,7 @@ private static InternalAggregationFunction generateAggregation(Type keyType, Typ new KeyValuePairsStateFactory(keyType, valueType), outputType); - GenericAccumulatorFactoryBinder factory = AccumulatorCompiler.generateAccumulatorFactoryBinder(metadata, classLoader); + GenericAccumulatorFactoryBinder factory = new AccumulatorCompiler().generateAccumulatorFactoryBinder(metadata, classLoader); return new InternalAggregationFunction(NAME, inputTypes, intermediateType, outputType, true, factory); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/MergeHyperLogLogAggregation.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/MergeHyperLogLogAggregation.java index 4c87d3f9ab61..13eb09127e39 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/MergeHyperLogLogAggregation.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/MergeHyperLogLogAggregation.java @@ -29,7 +29,7 @@ @AggregationFunction("merge") public final class MergeHyperLogLogAggregation { - private static final AccumulatorStateSerializer serializer = StateCompiler.generateStateSerializer(HyperLogLogState.class); + private static final AccumulatorStateSerializer serializer = new StateCompiler().generateStateSerializer(HyperLogLogState.class); private MergeHyperLogLogAggregation() {} diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/MultimapAggregationFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/MultimapAggregationFunction.java index 98de6aba9446..09a529a37da5 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/MultimapAggregationFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/MultimapAggregationFunction.java @@ -98,7 +98,7 @@ private static InternalAggregationFunction generateAggregation(Type keyType, Typ new KeyValuePairsStateFactory(keyType, valueType), outputType); - GenericAccumulatorFactoryBinder factory = AccumulatorCompiler.generateAccumulatorFactoryBinder(metadata, classLoader); + GenericAccumulatorFactoryBinder factory = new AccumulatorCompiler().generateAccumulatorFactoryBinder(metadata, classLoader); return new InternalAggregationFunction(NAME, inputTypes, intermediateType, outputType, true, factory); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/builder/HashAggregationBuilder.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/builder/HashAggregationBuilder.java index a4e63264030e..c0633d47407a 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/builder/HashAggregationBuilder.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/builder/HashAggregationBuilder.java @@ -16,7 +16,6 @@ import com.facebook.presto.spi.Page; import java.util.Iterator; -import java.util.concurrent.CompletableFuture; public interface HashAggregationBuilder extends AutoCloseable @@ -27,10 +26,6 @@ public interface HashAggregationBuilder boolean isFull(); - CompletableFuture isBlocked(); - - void updateMemory(); - @Override void close(); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/builder/InMemoryHashAggregationBuilder.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/builder/InMemoryHashAggregationBuilder.java index 6d325f82aaaf..f46a084ce9f1 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/builder/InMemoryHashAggregationBuilder.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/builder/InMemoryHashAggregationBuilder.java @@ -13,7 +13,6 @@ */ package com.facebook.presto.operator.aggregation.builder; -import com.facebook.presto.array.IntBigArray; import com.facebook.presto.memory.LocalMemoryContext; import com.facebook.presto.operator.GroupByHash; import com.facebook.presto.operator.GroupByIdBlock; @@ -24,28 +23,21 @@ import com.facebook.presto.spi.PageBuilder; import com.facebook.presto.spi.block.BlockBuilder; import com.facebook.presto.spi.type.Type; -import com.facebook.presto.sql.planner.plan.AggregationNode; import com.facebook.presto.sql.planner.plan.AggregationNode.Step; import com.google.common.collect.AbstractIterator; import com.google.common.collect.ImmutableList; import com.google.common.primitives.Ints; import io.airlift.units.DataSize; -import it.unimi.dsi.fastutil.ints.AbstractIntIterator; -import it.unimi.dsi.fastutil.ints.IntIterator; -import it.unimi.dsi.fastutil.ints.IntIterators; -import java.util.AbstractList; import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Optional; -import java.util.concurrent.CompletableFuture; import static com.facebook.presto.operator.GroupByHash.createGroupByHash; import static com.facebook.presto.spi.type.BigintType.BIGINT; import static com.google.common.base.Preconditions.checkArgument; import static java.util.Objects.requireNonNull; -import static java.util.concurrent.CompletableFuture.completedFuture; public class InMemoryHashAggregationBuilder implements HashAggregationBuilder @@ -68,28 +60,6 @@ public InMemoryHashAggregationBuilder( Optional hashChannel, OperatorContext operatorContext, DataSize maxPartialMemory) - { - this(accumulatorFactories, - step, - expectedGroups, - groupByTypes, - groupByChannels, - hashChannel, - operatorContext, - maxPartialMemory, - Optional.empty()); - } - - public InMemoryHashAggregationBuilder( - List accumulatorFactories, - AggregationNode.Step step, - int expectedGroups, - List groupByTypes, - List groupByChannels, - Optional hashChannel, - OperatorContext operatorContext, - DataSize maxPartialMemory, - Optional overwriteIntermediateChannelOffset) { this.groupByHash = createGroupByHash(operatorContext.getSession(), groupByTypes, Ints.toArray(groupByChannels), hashChannel, expectedGroups); this.operatorContext = operatorContext; @@ -102,11 +72,7 @@ public InMemoryHashAggregationBuilder( requireNonNull(accumulatorFactories, "accumulatorFactories is null"); for (int i = 0; i < accumulatorFactories.size(); i++) { AccumulatorFactory accumulatorFactory = accumulatorFactories.get(i); - Optional overwriteIntermediateChannel = Optional.empty(); - if (overwriteIntermediateChannelOffset.isPresent()) { - overwriteIntermediateChannel = Optional.of(overwriteIntermediateChannelOffset.get() + i); - } - builder.add(new Aggregator(accumulatorFactory, step, overwriteIntermediateChannel)); + builder.add(new Aggregator(accumulatorFactory, step)); } aggregators = builder.build(); } @@ -129,12 +95,15 @@ public void processPage(Page page) aggregator.processPage(groupIds, page); } } + updateMemory(); } - @Override - public void updateMemory() + private void updateMemory() { - long memorySize = getSizeInMemory(); + long memorySize = groupByHash.getEstimatedSize(); + for (Aggregator aggregator : aggregators) { + memorySize += aggregator.getEstimatedSize(); + } if (partial) { systemMemoryContext.setBytes(memorySize); full = (memorySize > maxPartialMemory); @@ -143,91 +112,38 @@ public void updateMemory() operatorContext.setMemoryReservation(memorySize); } } + @Override public boolean isFull() { return full; } - @Override - public CompletableFuture isBlocked() - { - return completedFuture(null); - } - - public long getSizeInMemory() - { - long sizeInMemory = groupByHash.getEstimatedSize(); - for (Aggregator aggregator : aggregators) { - sizeInMemory += aggregator.getEstimatedSize(); - } - return sizeInMemory; - } - - /** - * building hash sorted results requires memory for sorting group IDs. - * This method returns size of that memory requirement. - */ - public long getGroupIdsSortingSize() - { - return getGroupCount() * Integer.BYTES; - } - - public void setOutputPartial() - { - for (Aggregator aggregator : aggregators) { - aggregator.setOutputPartial(); - } - } - - public int getKeyChannels() - { - return groupByHash.getTypes().size(); - } - - public long getGroupCount() - { - return groupByHash.getGroupCount(); - } - @Override public Iterator buildResult() { - return buildResult(consecutiveGroupIds()); - } - - public Iterator buildHashSortedResult() - { - return buildResult(hashSortedGroupIds()); - } - - public List buildIntermediateTypes() - { - ArrayList types = new ArrayList<>(groupByHash.getTypes()); - for (InMemoryHashAggregationBuilder.Aggregator aggregator : aggregators) { - types.add(aggregator.getIntermediateType()); + List types = new ArrayList<>(groupByHash.getTypes()); + for (Aggregator aggregator : aggregators) { + types.add(aggregator.getType()); } - return types; - } - private Iterator buildResult(IntIterator groupIds) - { - final PageBuilder pageBuilder = new PageBuilder(buildTypes()); + final PageBuilder pageBuilder = new PageBuilder(types); return new AbstractIterator() { + private final int groupCount = groupByHash.getGroupCount(); + private int groupId; + @Override protected Page computeNext() { - if (!groupIds.hasNext()) { + if (groupId >= groupCount) { return endOfData(); } pageBuilder.reset(); List types = groupByHash.getTypes(); - while (!pageBuilder.isFull() && groupIds.hasNext()) { - int groupId = groupIds.nextInt(); - + while (!pageBuilder.isFull() && groupId < groupCount) { groupByHash.appendValuesTo(groupId, pageBuilder, 0); pageBuilder.declarePosition(); @@ -236,6 +152,8 @@ protected Page computeNext() BlockBuilder output = pageBuilder.getBlockBuilder(types.size() + i); aggregator.evaluate(groupId, output); } + + groupId++; } return pageBuilder.build(); @@ -243,91 +161,18 @@ protected Page computeNext() }; } - public List buildTypes() - { - ArrayList types = new ArrayList<>(groupByHash.getTypes()); - for (Aggregator aggregator : aggregators) { - types.add(aggregator.getType()); - } - return types; - } - - private IntIterator consecutiveGroupIds() - { - return IntIterators.fromTo(0, groupByHash.getGroupCount()); - } - - private IntIterator hashSortedGroupIds() - { - IntBigArray groupIds = new IntBigArray(); - groupIds.ensureCapacity(groupByHash.getGroupCount()); - for (int i = 0; i < groupByHash.getGroupCount(); i++) { - groupIds.set(i, i); - } - - List wrappedGroupIds = asList(groupIds, groupByHash.getGroupCount()); - wrappedGroupIds.sort(((leftGroupId, rightGroupId) -> - Long.compare(groupByHash.getRawHash(leftGroupId), groupByHash.getRawHash(rightGroupId)))); - - return new AbstractIntIterator() { - private final int totalPositions = groupByHash.getGroupCount(); - private int position = 0; - - @Override - public boolean hasNext() - { - return position < totalPositions; - } - - @Override - public int nextInt() - { - return groupIds.get(position++); - } - }; - } - - private static List asList(IntBigArray groupIds, int size) - { - return new AbstractList() { - @Override - public Integer get(int index) - { - return groupIds.get(index); - } - - @Override - public int size() - { - return size; - } - - @Override - public Integer set(int index, Integer element) - { - int oldValue = groupIds.get(index); - groupIds.set(index, element); - return oldValue; - } - }; - } - private static class Aggregator { private final GroupedAccumulator aggregation; - private AggregationNode.Step step; + private final Step step; private final int intermediateChannel; - private Aggregator(AccumulatorFactory accumulatorFactory, AggregationNode.Step step, Optional overwriteIntermediateChannel) + private Aggregator(AccumulatorFactory accumulatorFactory, Step step) { if (step.isInputRaw()) { this.intermediateChannel = -1; this.aggregation = accumulatorFactory.createGroupedAccumulator(); } - else if (overwriteIntermediateChannel.isPresent()) { - this.intermediateChannel = overwriteIntermediateChannel.get(); - this.aggregation = accumulatorFactory.createGroupedIntermediateAccumulator(); - } else { checkArgument(accumulatorFactory.getInputChannels().size() == 1, "expected 1 input channel for intermediate aggregation"); this.intermediateChannel = accumulatorFactory.getInputChannels().get(0); @@ -370,16 +215,6 @@ public void evaluate(int groupId, BlockBuilder output) aggregation.evaluateFinal(groupId, output); } } - - public void setOutputPartial() - { - step = AggregationNode.Step.partialOutput(step); - } - - public Type getIntermediateType() - { - return aggregation.getIntermediateType(); - } } public static List toTypes(List groupByType, Step step, List factories, Optional hashChannel) @@ -390,7 +225,7 @@ public static List toTypes(List groupByType, Step step, Li types.add(BIGINT); } for (AccumulatorFactory factory : factories) { - types.add(new Aggregator(factory, step, Optional.empty()).getType()); + types.add(new Aggregator(factory, step).getType()); } return types.build(); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/builder/MergingHashAggregationBuilder.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/builder/MergingHashAggregationBuilder.java deleted file mode 100644 index 0108e1d96de2..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/builder/MergingHashAggregationBuilder.java +++ /dev/null @@ -1,136 +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.operator.aggregation.builder; - -import com.facebook.presto.memory.LocalMemoryContext; -import com.facebook.presto.operator.OperatorContext; -import com.facebook.presto.operator.aggregation.AccumulatorFactory; -import com.facebook.presto.spi.Page; -import com.facebook.presto.spi.type.Type; -import com.facebook.presto.sql.planner.plan.AggregationNode; -import com.google.common.collect.ImmutableList; -import io.airlift.units.DataSize; - -import java.io.Closeable; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.Optional; - -public class MergingHashAggregationBuilder - implements Closeable -{ - private final List accumulatorFactories; - private final AggregationNode.Step step; - private final int expectedGroups; - private final ImmutableList groupByPartialChannels; - private final Optional hashChannel; - private final OperatorContext operatorContext; - private final Iterator sortedPages; - private InMemoryHashAggregationBuilder hashAggregationBuilder; - private final List groupByTypes; - private final LocalMemoryContext systemMemoryContext; - private final long memorySizeBeforeSpill; - private final int overwriteIntermediateChannelOffset; - - public MergingHashAggregationBuilder( - List accumulatorFactories, - AggregationNode.Step step, - int expectedGroups, - List groupByTypes, - Optional hashChannel, - OperatorContext operatorContext, - Iterator sortedPages, - LocalMemoryContext systemMemoryContext, - long memorySizeBeforeSpill, - int overwriteIntermediateChannelOffset) - { - ImmutableList.Builder groupByPartialChannels = ImmutableList.builder(); - for (int i = 0; i < groupByTypes.size(); i++) { - groupByPartialChannels.add(i); - } - - this.accumulatorFactories = accumulatorFactories; - this.step = AggregationNode.Step.partialInput(step); - this.expectedGroups = expectedGroups; - this.groupByPartialChannels = groupByPartialChannels.build(); - this.hashChannel = hashChannel.isPresent() ? Optional.of(groupByTypes.size()) : hashChannel; - this.operatorContext = operatorContext; - this.sortedPages = sortedPages; - this.groupByTypes = groupByTypes; - this.systemMemoryContext = systemMemoryContext; - this.memorySizeBeforeSpill = memorySizeBeforeSpill; - this.overwriteIntermediateChannelOffset = overwriteIntermediateChannelOffset; - - rebuildHashAggregationBuilder(); - } - - public Iterator buildResult() - { - return new Iterator() { - private Iterator resultPages = Collections.emptyIterator(); - - @Override - public boolean hasNext() - { - return sortedPages.hasNext() || resultPages.hasNext(); - } - - @Override - public Page next() - { - if (!resultPages.hasNext()) { - rebuildHashAggregationBuilder(); - long memorySize = 0; // ensure that at least one merged page will be processed - - // we can produce output after every page, because sortedPages does not have - // hash values that span multiple pages (guaranteed by MergeHashSort) - while (sortedPages.hasNext() && !shouldProduceOutput(memorySize)) { - hashAggregationBuilder.processPage(sortedPages.next()); - memorySize = hashAggregationBuilder.getSizeInMemory(); - systemMemoryContext.setBytes(memorySize); - } - resultPages = hashAggregationBuilder.buildResult(); - } - - return resultPages.next(); - } - }; - } - - @Override - public void close() - { - hashAggregationBuilder.close(); - } - - private boolean shouldProduceOutput(long memorySize) - { - return (memorySizeBeforeSpill > 0 && memorySize > memorySizeBeforeSpill); - } - - private void rebuildHashAggregationBuilder() - { - this.hashAggregationBuilder = new InMemoryHashAggregationBuilder( - accumulatorFactories, - step, - expectedGroups, - groupByTypes, - groupByPartialChannels, - hashChannel, - operatorContext, - DataSize.succinctBytes(0), - Optional.of(overwriteIntermediateChannelOffset)); - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/builder/SpillableHashAggregationBuilder.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/builder/SpillableHashAggregationBuilder.java deleted file mode 100644 index 352952e7c770..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/builder/SpillableHashAggregationBuilder.java +++ /dev/null @@ -1,273 +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.operator.aggregation.builder; - -import com.facebook.presto.memory.AbstractAggregatedMemoryContext; -import com.facebook.presto.memory.LocalMemoryContext; -import com.facebook.presto.operator.MergeHashSort; -import com.facebook.presto.operator.OperatorContext; -import com.facebook.presto.operator.aggregation.AccumulatorFactory; -import com.facebook.presto.spi.Page; -import com.facebook.presto.spi.type.Type; -import com.facebook.presto.spiller.Spiller; -import com.facebook.presto.spiller.SpillerFactory; -import com.facebook.presto.sql.planner.plan.AggregationNode; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; -import io.airlift.units.DataSize; - -import java.util.Iterator; -import java.util.List; -import java.util.Optional; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; - -import static com.google.common.base.Preconditions.checkState; -import static io.airlift.concurrent.MoreFutures.getFutureValue; -import static java.lang.Math.max; - -public class SpillableHashAggregationBuilder - implements HashAggregationBuilder -{ - private InMemoryHashAggregationBuilder hashAggregationBuilder; - private final SpillerFactory spillerFactory; - private final List accumulatorFactories; - private final AggregationNode.Step step; - private final int expectedGroups; - private final List groupByTypes; - private final List groupByChannels; - private final Optional hashChannel; - private final OperatorContext operatorContext; - private final long memorySizeBeforeSpill; - private final long memoryLimitForMergeWithMemory; - private Optional spiller = Optional.empty(); - private Optional merger = Optional.empty(); - private CompletableFuture spillInProgress = CompletableFuture.completedFuture(null); - private final LocalMemoryContext aggregationMemoryContext; - private final LocalMemoryContext spillMemoryContext; - - public SpillableHashAggregationBuilder( - List accumulatorFactories, - AggregationNode.Step step, - int expectedGroups, - List groupByTypes, - List groupByChannels, - Optional hashChannel, - OperatorContext operatorContext, - DataSize memoryLimitBeforeSpill, - DataSize memoryLimitForMergeWithMemory, - SpillerFactory spillerFactory) - { - this.accumulatorFactories = accumulatorFactories; - this.step = step; - this.expectedGroups = expectedGroups; - this.groupByTypes = groupByTypes; - this.groupByChannels = groupByChannels; - this.hashChannel = hashChannel; - this.operatorContext = operatorContext; - this.memorySizeBeforeSpill = memoryLimitBeforeSpill.toBytes(); - this.memoryLimitForMergeWithMemory = memoryLimitForMergeWithMemory.toBytes(); - this.spillerFactory = spillerFactory; - - AbstractAggregatedMemoryContext systemMemoryContext = operatorContext.getSystemMemoryContext(); - this.aggregationMemoryContext = systemMemoryContext.newLocalMemoryContext(); - this.spillMemoryContext = systemMemoryContext.newLocalMemoryContext(); - - rebuildHashAggregationBuilder(); - } - - @Override - public void processPage(Page page) - { - checkState(hasPreviousSpillCompletedSuccessfully(), "Previous spill hasn't yet finished"); - - hashAggregationBuilder.processPage(page); - - if (shouldSpill(getSizeInMemory())) { - spillToDisk(); - } - } - - @Override - public void updateMemory() - { - aggregationMemoryContext.setBytes(getSizeInMemory()); - - if (spillInProgress.isDone()) { - spillMemoryContext.setBytes(0L); - } - } - - public long getSizeInMemory() - { - // TODO: we could skip memory reservation for hashAggregationBuilder.getGroupIdsSortingSize() - // if before building result from hashAggregationBuilder we would convert it to "read only" version. - // Read only version of GroupByHash from hashAggregationBuilder could be compacted by dropping - // most of it's field, freeing up some memory that could be used for sorting. - return hashAggregationBuilder.getSizeInMemory() + hashAggregationBuilder.getGroupIdsSortingSize(); - } - - @Override - public boolean isFull() - { - return false; - } - - @Override - public CompletableFuture isBlocked() - { - return spillInProgress; - } - - private boolean hasPreviousSpillCompletedSuccessfully() - { - if (isBlocked().isDone()) { - // check for exception from previous spill for early failure - getFutureValue(spillInProgress); - return true; - } - else { - return false; - } - } - - private boolean shouldSpill(long memorySize) - { - return (memorySizeBeforeSpill > 0 && memorySize > memorySizeBeforeSpill); - } - - private boolean shouldMergeWithMemory(long memorySize) - { - return memorySize < memoryLimitForMergeWithMemory; - } - - @Override - public Iterator buildResult() - { - checkState(hasPreviousSpillCompletedSuccessfully(), "Previous spill hasn't yet finished"); - - if (!spiller.isPresent()) { - return hashAggregationBuilder.buildResult(); - } - - try { - if (shouldMergeWithMemory(getSizeInMemory())) { - return mergeFromDiskAndMemory(); - } - else { - spillToDisk().get(); - return mergeFromDisk(); - } - } - catch (InterruptedException | ExecutionException e) { - Thread.currentThread().interrupt(); - throw Throwables.propagate(e); - } - } - - @Override - public void close() - { - if (merger.isPresent()) { - merger.get().close(); - } - if (spiller.isPresent()) { - spiller.get().close(); - } - } - - private CompletableFuture spillToDisk() - { - checkState(hasPreviousSpillCompletedSuccessfully(), "Previous spill hasn't yet finished"); - hashAggregationBuilder.setOutputPartial(); - - if (!spiller.isPresent()) { - spiller = Optional.of(spillerFactory.create(hashAggregationBuilder.buildTypes())); - } - long spillMemoryUsage = getSizeInMemory(); - - // start spilling process with current content of the hashAggregationBuilder builder... - spillInProgress = spiller.get().spill(hashAggregationBuilder.buildHashSortedResult()); - // ... and immediately create new hashAggregationBuilder so effectively memory ownership - // over hashAggregationBuilder is transferred from this thread to a spilling thread - rebuildHashAggregationBuilder(); - - // First decrease memory usage of aggregation context... - aggregationMemoryContext.setBytes(getSizeInMemory()); - // And then transfer this memory to spill context - // TODO: is there an easy way to do this atomically? - spillMemoryContext.setBytes(spillMemoryUsage); - - return spillInProgress; - } - - private Iterator mergeFromDiskAndMemory() - { - checkState(spiller.isPresent()); - - hashAggregationBuilder.setOutputPartial(); - - Iterator mergedSpilledPages = MergeHashSort.merge( - groupByTypes, - hashAggregationBuilder.buildIntermediateTypes(), - ImmutableList.>builder() - .addAll(spiller.get().getSpills()) - .add(hashAggregationBuilder.buildHashSortedResult()) - .build()); - - return mergeSortedPages(mergedSpilledPages, max(memorySizeBeforeSpill - memoryLimitForMergeWithMemory, 1L)); - } - - private Iterator mergeFromDisk() - { - checkState(spiller.isPresent()); - - Iterator mergedSpilledPages = MergeHashSort.merge( - groupByTypes, - hashAggregationBuilder.buildIntermediateTypes(), - spiller.get().getSpills()); - - return mergeSortedPages(mergedSpilledPages, memorySizeBeforeSpill); - } - - private Iterator mergeSortedPages(Iterator sortedPages, long memorySizeBeforeSpill) - { - merger = Optional.of(new MergingHashAggregationBuilder( - accumulatorFactories, - step, - expectedGroups, - groupByTypes, - hashChannel, - operatorContext, - sortedPages, - operatorContext.getSystemMemoryContext().newLocalMemoryContext(), - memorySizeBeforeSpill, - hashAggregationBuilder.getKeyChannels())); - - return merger.get().buildResult(); - } - - private void rebuildHashAggregationBuilder() - { - this.hashAggregationBuilder = new InMemoryHashAggregationBuilder( - accumulatorFactories, - step, - expectedGroups, - groupByTypes, - groupByChannels, - hashChannel, - operatorContext, - DataSize.succinctBytes(0)); - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/state/StateCompiler.java b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/state/StateCompiler.java index c5e2d6cfa090..91251f8da9c6 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/aggregation/state/StateCompiler.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/aggregation/state/StateCompiler.java @@ -89,10 +89,6 @@ public class StateCompiler { - private StateCompiler() - { - } - private static Class getBigArrayType(Class type) { if (type.equals(long.class)) { @@ -117,12 +113,12 @@ private static Class getBigArrayType(Class type) throw new IllegalArgumentException("Unsupported type: " + type.getName()); } - public static AccumulatorStateSerializer generateStateSerializer(Class clazz) + public AccumulatorStateSerializer generateStateSerializer(Class clazz) { return generateStateSerializer(clazz, new DynamicClassLoader(clazz.getClassLoader())); } - public static AccumulatorStateSerializer generateStateSerializer(Class clazz, DynamicClassLoader classLoader) + public AccumulatorStateSerializer generateStateSerializer(Class clazz, DynamicClassLoader classLoader) { AccumulatorStateMetadata metadata = getMetadataAnnotation(clazz); if (metadata != null && metadata.stateSerializerClass() != void.class) { @@ -318,12 +314,12 @@ private static Method getGetter(Class clazz, StateField field) } } - public static AccumulatorStateFactory generateStateFactory(Class clazz) + public AccumulatorStateFactory generateStateFactory(Class clazz) { return generateStateFactory(clazz, new DynamicClassLoader(clazz.getClassLoader())); } - public static AccumulatorStateFactory generateStateFactory(Class clazz, DynamicClassLoader classLoader) + public AccumulatorStateFactory generateStateFactory(Class clazz, DynamicClassLoader classLoader) { AccumulatorStateMetadata metadata = getMetadataAnnotation(clazz); if (metadata != null && metadata.stateFactoryClass() != void.class) { @@ -541,8 +537,8 @@ private static FieldDefinition generateGroupedField(ClassDefinition definition, private static List enumerateFields(Class clazz) { ImmutableList.Builder builder = ImmutableList.builder(); - final Set> primitiveClasses = ImmutableSet.of(byte.class, boolean.class, long.class, double.class); - Set> supportedClasses = ImmutableSet.of(byte.class, boolean.class, long.class, double.class, Slice.class, Block.class); + final Set> primitiveClasses = ImmutableSet.>of(byte.class, boolean.class, long.class, double.class); + Set> supportedClasses = ImmutableSet.>of(byte.class, boolean.class, long.class, double.class, Slice.class, Block.class); for (Method method : clazz.getMethods()) { if (method.getName().equals("getEstimatedSize")) { diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ApplyFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ApplyFunction.java deleted file mode 100644 index 711f48ce6719..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ApplyFunction.java +++ /dev/null @@ -1,97 +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.operator.scalar; - -import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionKind; -import com.facebook.presto.metadata.FunctionRegistry; -import com.facebook.presto.metadata.Signature; -import com.facebook.presto.metadata.SqlScalarFunction; -import com.facebook.presto.spi.type.Type; -import com.facebook.presto.spi.type.TypeManager; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; - -import java.lang.invoke.MethodHandle; - -import static com.facebook.presto.metadata.Signature.typeVariable; -import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature; -import static com.facebook.presto.util.Reflection.methodHandle; -import static com.google.common.primitives.Primitives.wrap; - -/** - * This scalar function exists primarily to test lambda expression support. - */ -public final class ApplyFunction - extends SqlScalarFunction -{ - public static final ApplyFunction APPLY_FUNCTION = new ApplyFunction(); - - private static final MethodHandle METHOD_HANDLE = methodHandle(ApplyFunction.class, "apply", Object.class, MethodHandle.class); - - private ApplyFunction() - { - super(new Signature( - "apply", - FunctionKind.SCALAR, - ImmutableList.of(typeVariable("T"), typeVariable("U")), - ImmutableList.of(), - parseTypeSignature("U"), - ImmutableList.of(parseTypeSignature("T"), parseTypeSignature("function(T,U)")), - false)); - } - - @Override - public boolean isHidden() - { - return true; - } - - @Override - public boolean isDeterministic() - { - return true; - } - - @Override - public String getDescription() - { - return "lambda apply function"; - } - - @Override - public ScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionRegistry functionRegistry) - { - Type argumentType = boundVariables.getTypeVariable("T"); - Type returnType = boundVariables.getTypeVariable("U"); - return new ScalarFunctionImplementation( - true, - ImmutableList.of(true, false), - METHOD_HANDLE.asType( - METHOD_HANDLE.type() - .changeReturnType(wrap(returnType.getJavaType())) - .changeParameterType(0, wrap(argumentType.getJavaType()))), - isDeterministic()); - } - - public static Object apply(Object input, MethodHandle function) - { - try { - return function.invoke(input); - } - catch (Throwable throwable) { - throw Throwables.propagate(throwable); - } - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayElementAtFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayElementAtFunction.java index 786dfb5cefaf..217641d58e1a 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayElementAtFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayElementAtFunction.java @@ -47,9 +47,6 @@ public static Void voidElementAt(@SqlType("array(E)") Block array, @SqlType("big public static Long longElementAt(@TypeParameter("E") Type elementType, @SqlType("array(E)") Block array, @SqlType("bigint") long index) { int position = checkedIndexToBlockPosition(array, index); - if (position == -1) { - return null; - } if (array.isNull(position)) { return null; } @@ -63,9 +60,6 @@ public static Long longElementAt(@TypeParameter("E") Type elementType, @SqlType( public static Boolean booleanElementAt(@TypeParameter("E") Type elementType, @SqlType("array(E)") Block array, @SqlType("bigint") long index) { int position = checkedIndexToBlockPosition(array, index); - if (position == -1) { - return null; - } if (array.isNull(position)) { return null; } @@ -79,9 +73,6 @@ public static Boolean booleanElementAt(@TypeParameter("E") Type elementType, @Sq public static Double doubleElementAt(@TypeParameter("E") Type elementType, @SqlType("array(E)") Block array, @SqlType("bigint") long index) { int position = checkedIndexToBlockPosition(array, index); - if (position == -1) { - return null; - } if (array.isNull(position)) { return null; } @@ -95,9 +86,6 @@ public static Double doubleElementAt(@TypeParameter("E") Type elementType, @SqlT public static Slice sliceElementAt(@TypeParameter("E") Type elementType, @SqlType("array(E)") Block array, @SqlType("bigint") long index) { int position = checkedIndexToBlockPosition(array, index); - if (position == -1) { - return null; - } if (array.isNull(position)) { return null; } @@ -111,9 +99,6 @@ public static Slice sliceElementAt(@TypeParameter("E") Type elementType, @SqlTyp public static Block blockElementAt(@TypeParameter("E") Type elementType, @SqlType("array(E)") Block array, @SqlType("bigint") long index) { int position = checkedIndexToBlockPosition(array, index); - if (position == -1) { - return null; - } if (array.isNull(position)) { return null; } @@ -121,9 +106,6 @@ public static Block blockElementAt(@TypeParameter("E") Type elementType, @SqlTyp return (Block) elementType.getObject(array, position); } - /** - * @return PrestoException if the index is 0, -1 if the index is out of range (to tell the calling function to return null), and the element position otherwise. - */ private static int checkedIndexToBlockPosition(Block block, long index) { int arrayLength = block.getPositionCount(); @@ -131,8 +113,9 @@ private static int checkedIndexToBlockPosition(Block block, long index) throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "SQL array indices start at 1"); } if (Math.abs(index) > arrayLength) { - return -1; // -1 indicates that the element is out of range and "ELEMENT_AT" should return null + throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "Array subscript out of bounds"); } + if (index > 0) { return Ints.checkedCast(index - 1); } diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayFilterFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayFilterFunction.java deleted file mode 100644 index c91653e31545..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayFilterFunction.java +++ /dev/null @@ -1,105 +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.operator.scalar; - -import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionKind; -import com.facebook.presto.metadata.FunctionRegistry; -import com.facebook.presto.metadata.Signature; -import com.facebook.presto.metadata.SqlScalarFunction; -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.Type; -import com.facebook.presto.spi.type.TypeManager; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; - -import java.lang.invoke.MethodHandle; - -import static com.facebook.presto.metadata.Signature.typeVariable; -import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature; -import static com.facebook.presto.spi.type.TypeUtils.readNativeValue; -import static com.facebook.presto.util.Reflection.methodHandle; -import static java.lang.Boolean.TRUE; - -public final class ArrayFilterFunction - extends SqlScalarFunction -{ - public static final ArrayFilterFunction ARRAY_FILTER_FUNCTION = new ArrayFilterFunction(); - - private static final MethodHandle METHOD_HANDLE = methodHandle(ArrayFilterFunction.class, "filter", Type.class, Block.class, MethodHandle.class); - - private ArrayFilterFunction() - { - super(new Signature( - "filter", - FunctionKind.SCALAR, - ImmutableList.of(typeVariable("T")), - ImmutableList.of(), - parseTypeSignature("array(T)"), - ImmutableList.of(parseTypeSignature("array(T)"), parseTypeSignature("function(T,boolean)")), - false)); - } - - @Override - public boolean isHidden() - { - return false; - } - - @Override - public boolean isDeterministic() - { - return false; - } - - @Override - public String getDescription() - { - return "return array containing elements that match the given predicate"; - } - - @Override - public ScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionRegistry functionRegistry) - { - Type type = boundVariables.getTypeVariable("T"); - return new ScalarFunctionImplementation( - false, - ImmutableList.of(false, false), - METHOD_HANDLE.bindTo(type), - isDeterministic()); - } - - public static Block filter(Type type, Block block, MethodHandle function) - { - int positionCount = block.getPositionCount(); - BlockBuilder resultBuilder = type.createBlockBuilder(new BlockBuilderStatus(), positionCount); - for (int position = 0; position < positionCount; position++) { - Object input = readNativeValue(type, block, position); - Boolean keep; - try { - keep = (Boolean) function.invoke(input); - } - catch (Throwable throwable) { - throw Throwables.propagate(throwable); - } - if (TRUE.equals(keep)) { - type.appendTo(block, position, resultBuilder); - } - } - return resultBuilder.build(); - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayReduceFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayReduceFunction.java deleted file mode 100644 index 65bde4033417..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayReduceFunction.java +++ /dev/null @@ -1,115 +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.operator.scalar; - -import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionKind; -import com.facebook.presto.metadata.FunctionRegistry; -import com.facebook.presto.metadata.Signature; -import com.facebook.presto.metadata.SqlScalarFunction; -import com.facebook.presto.spi.block.Block; -import com.facebook.presto.spi.type.Type; -import com.facebook.presto.spi.type.TypeManager; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; -import com.google.common.primitives.Primitives; - -import java.lang.invoke.MethodHandle; - -import static com.facebook.presto.metadata.Signature.typeVariable; -import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature; -import static com.facebook.presto.spi.type.TypeUtils.readNativeValue; -import static com.facebook.presto.util.Reflection.methodHandle; - -public final class ArrayReduceFunction - extends SqlScalarFunction -{ - public static final ArrayReduceFunction ARRAY_REDUCE_FUNCTION = new ArrayReduceFunction(); - - private static final MethodHandle METHOD_HANDLE = methodHandle(ArrayReduceFunction.class, "reduce", Type.class, Block.class, Object.class, MethodHandle.class, MethodHandle.class); - - private ArrayReduceFunction() - { - super(new Signature( - "reduce", - FunctionKind.SCALAR, - ImmutableList.of(typeVariable("T"), typeVariable("S"), typeVariable("R")), - ImmutableList.of(), - parseTypeSignature("R"), - ImmutableList.of(parseTypeSignature("array(T)"), parseTypeSignature("S"), parseTypeSignature("function(S,T,S)"), parseTypeSignature("function(S,R)")), - false)); - } - - @Override - public boolean isHidden() - { - return false; - } - - @Override - public boolean isDeterministic() - { - return false; - } - - @Override - public String getDescription() - { - return "Reduce elements of the array into a single value"; - } - - @Override - public ScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionRegistry functionRegistry) - { - Type inputType = boundVariables.getTypeVariable("T"); - Type intermediateType = boundVariables.getTypeVariable("S"); - Type outputType = boundVariables.getTypeVariable("R"); - MethodHandle methodHandle = METHOD_HANDLE.bindTo(inputType); - return new ScalarFunctionImplementation( - true, - ImmutableList.of(false, true, false, false), - methodHandle.asType( - methodHandle.type() - .changeParameterType(1, Primitives.wrap(intermediateType.getJavaType())) - .changeReturnType(Primitives.wrap(outputType.getJavaType()))), - isDeterministic()); - } - - public static Object reduce( - Type inputType, - Block block, - Object initialIntermediateValue, - MethodHandle inputFunction, - MethodHandle outputFunction) - { - int positionCount = block.getPositionCount(); - Object intermediateValue = initialIntermediateValue; - for (int position = 0; position < positionCount; position++) { - Object input = readNativeValue(inputType, block, position); - try { - intermediateValue = inputFunction.invoke(intermediateValue, input); - } - catch (Throwable throwable) { - throw Throwables.propagate(throwable); - } - } - try { - return outputFunction.invoke(intermediateValue); - } - catch (Throwable throwable) { - throw Throwables.propagate(throwable); - } - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayTransformFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayTransformFunction.java deleted file mode 100644 index c722d098668b..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/ArrayTransformFunction.java +++ /dev/null @@ -1,104 +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.operator.scalar; - -import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionKind; -import com.facebook.presto.metadata.FunctionRegistry; -import com.facebook.presto.metadata.Signature; -import com.facebook.presto.metadata.SqlScalarFunction; -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.Type; -import com.facebook.presto.spi.type.TypeManager; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; - -import java.lang.invoke.MethodHandle; - -import static com.facebook.presto.metadata.Signature.typeVariable; -import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature; -import static com.facebook.presto.spi.type.TypeUtils.readNativeValue; -import static com.facebook.presto.spi.type.TypeUtils.writeNativeValue; -import static com.facebook.presto.util.Reflection.methodHandle; - -public final class ArrayTransformFunction - extends SqlScalarFunction -{ - public static final ArrayTransformFunction ARRAY_TRANSFORM_FUNCTION = new ArrayTransformFunction(); - - private static final MethodHandle METHOD_HANDLE = methodHandle(ArrayTransformFunction.class, "transform", Type.class, Type.class, Block.class, MethodHandle.class); - - private ArrayTransformFunction() - { - super(new Signature( - "transform", - FunctionKind.SCALAR, - ImmutableList.of(typeVariable("T"), typeVariable("U")), - ImmutableList.of(), - parseTypeSignature("array(U)"), - ImmutableList.of(parseTypeSignature("array(T)"), parseTypeSignature("function(T,U)")), - false)); - } - - @Override - public boolean isHidden() - { - return false; - } - - @Override - public boolean isDeterministic() - { - return false; - } - - @Override - public String getDescription() - { - return "apply lambda to each element of the array"; - } - - @Override - public ScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionRegistry functionRegistry) - { - Type inputType = boundVariables.getTypeVariable("T"); - Type outputType = boundVariables.getTypeVariable("U"); - return new ScalarFunctionImplementation( - false, - ImmutableList.of(false, false), - METHOD_HANDLE.bindTo(inputType).bindTo(outputType), - isDeterministic()); - } - - public static Block transform(Type inputType, Type outputType, Block block, MethodHandle function) - { - int positionCount = block.getPositionCount(); - BlockBuilder resultBuilder = outputType.createBlockBuilder(new BlockBuilderStatus(), positionCount); - for (int position = 0; position < positionCount; position++) { - Object input = readNativeValue(inputType, block, position); - Object output; - try { - output = function.invoke(input); - } - catch (Throwable throwable) { - throw Throwables.propagate(throwable); - } - writeNativeValue(outputType, resultBuilder, output); - } - return resultBuilder.build(); - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/EmptyMapConstructor.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/EmptyMapConstructor.java deleted file mode 100644 index 54c4ddc3bc8c..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/EmptyMapConstructor.java +++ /dev/null @@ -1,39 +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.operator.scalar; - -import com.facebook.presto.spi.block.Block; -import com.facebook.presto.spi.block.BlockBuilderStatus; -import com.facebook.presto.spi.block.InterleavedBlockBuilder; -import com.facebook.presto.spi.function.Description; -import com.facebook.presto.spi.function.ScalarFunction; -import com.facebook.presto.spi.function.SqlType; -import com.google.common.collect.ImmutableList; - -import static com.facebook.presto.type.UnknownType.UNKNOWN; - -public final class EmptyMapConstructor -{ - private static final Block EMPTY_MAP = new InterleavedBlockBuilder(ImmutableList.of(UNKNOWN, UNKNOWN), new BlockBuilderStatus(), 0).build(); - - private EmptyMapConstructor() {} - - @Description("Creates an empty map") - @ScalarFunction - @SqlType("map(unknown,unknown)") - public static Block map() - { - return EMPTY_MAP; - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/FailureFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/FailureFunction.java index 75af69da8120..5864482a8cdd 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/FailureFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/FailureFunction.java @@ -31,7 +31,7 @@ private FailureFunction() {} // We shouldn't be using UNKNOWN as an explicit type. This will be fixed when we fix type inference @Description("Decodes json to an exception and throws it") - @ScalarFunction(hidden = true) + @ScalarFunction @SqlType("unknown") public static void fail(@SqlType(StandardTypes.JSON) Slice failureInfoSlice) { diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapConcatFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapConcatFunction.java index fbb9b172d24b..24e83bdddee1 100644 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapConcatFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapConcatFunction.java @@ -40,13 +40,6 @@ public static Block mapConcat( @SqlType("map(K,V)") Block leftMap, @SqlType("map(K,V)") Block rightMap) { - if (leftMap.getPositionCount() == 0) { - return rightMap; - } - if (rightMap.getPositionCount() == 0) { - return leftMap; - } - TypedSet typedSet = new TypedSet(keyType, rightMap.getPositionCount()); BlockBuilder blockBuilder = new InterleavedBlockBuilder(ImmutableList.of(keyType, valueType), new BlockBuilderStatus(), leftMap.getPositionCount() + rightMap.getPositionCount()); for (int i = 0; i < rightMap.getPositionCount(); i += 2) { diff --git a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapFilterFunction.java b/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapFilterFunction.java deleted file mode 100644 index 46bf025aa663..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/operator/scalar/MapFilterFunction.java +++ /dev/null @@ -1,109 +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.operator.scalar; - -import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionKind; -import com.facebook.presto.metadata.FunctionRegistry; -import com.facebook.presto.metadata.Signature; -import com.facebook.presto.metadata.SqlScalarFunction; -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.TypeManager; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; - -import java.lang.invoke.MethodHandle; - -import static com.facebook.presto.metadata.Signature.typeVariable; -import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature; -import static com.facebook.presto.spi.type.TypeUtils.readNativeValue; -import static com.facebook.presto.util.Reflection.methodHandle; -import static java.lang.Boolean.TRUE; - -public final class MapFilterFunction - extends SqlScalarFunction -{ - public static final MapFilterFunction MAP_FILTER_FUNCTION = new MapFilterFunction(); - - private static final MethodHandle METHOD_HANDLE = methodHandle(MapFilterFunction.class, "filter", Type.class, Type.class, Block.class, MethodHandle.class); - - private MapFilterFunction() - { - super(new Signature( - "map_filter", - FunctionKind.SCALAR, - ImmutableList.of(typeVariable("K"), typeVariable("V")), - ImmutableList.of(), - parseTypeSignature("map(K,V)"), - ImmutableList.of(parseTypeSignature("map(K,V)"), parseTypeSignature("function(K,V,boolean)")), - false)); - } - - @Override - public boolean isHidden() - { - return false; - } - - @Override - public boolean isDeterministic() - { - return false; - } - - @Override - public String getDescription() - { - return "return map containing entries that match the given predicate"; - } - - @Override - public ScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionRegistry functionRegistry) - { - Type keyType = boundVariables.getTypeVariable("K"); - Type valueType = boundVariables.getTypeVariable("V"); - return new ScalarFunctionImplementation( - false, - ImmutableList.of(false, false), - METHOD_HANDLE.bindTo(keyType).bindTo(valueType), - isDeterministic()); - } - - public static Block filter(Type keyType, Type valueType, Block block, MethodHandle function) - { - int positionCount = block.getPositionCount(); - BlockBuilder resultBuilder = new InterleavedBlockBuilder(ImmutableList.of(keyType, valueType), new BlockBuilderStatus(), positionCount); - for (int position = 0; position < positionCount; position += 2) { - Object key = readNativeValue(keyType, block, position); - Object value = readNativeValue(valueType, block, position + 1); - Boolean keep; - try { - keep = (Boolean) function.invoke(key, value); - } - catch (Throwable throwable) { - throw Throwables.propagate(throwable); - } - if (TRUE.equals(keep)) { - keyType.appendTo(block, position, resultBuilder); - valueType.appendTo(block, position + 1, resultBuilder); - } - } - return resultBuilder.build(); - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/server/CoordinatorModule.java b/presto-main/src/main/java/com/facebook/presto/server/CoordinatorModule.java index fa56b5aeb713..b5abdaa641ca 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/CoordinatorModule.java +++ b/presto-main/src/main/java/com/facebook/presto/server/CoordinatorModule.java @@ -141,6 +141,15 @@ protected void setup(Binder binder) jsonCodecBinder(binder).bindJsonCodec(QueryResults.class); jaxrsBinder(binder).bind(StatementResource.class); + // execute resource + jaxrsBinder(binder).bind(ExecuteResource.class); + httpClientBinder(binder).bindHttpClient("execute", ForExecute.class) + .withTracing() + .withConfigDefaults(config -> { + config.setIdleTimeout(new Duration(30, SECONDS)); + config.setRequestTimeout(new Duration(10, SECONDS)); + }); + // query execution visualizer jaxrsBinder(binder).bind(QueryExecutionResource.class); diff --git a/presto-main/src/main/java/com/facebook/presto/server/ExecuteResource.java b/presto-main/src/main/java/com/facebook/presto/server/ExecuteResource.java new file mode 100644 index 000000000000..8b7743a680d4 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/server/ExecuteResource.java @@ -0,0 +1,178 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.server; + +import com.facebook.presto.client.ClientSession; +import com.facebook.presto.client.Column; +import com.facebook.presto.client.QueryResults; +import com.facebook.presto.client.StatementClient; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.AbstractIterator; +import io.airlift.http.client.HttpClient; +import io.airlift.http.server.HttpServerInfo; +import io.airlift.json.JsonCodec; +import io.airlift.units.Duration; + +import javax.inject.Inject; +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.WebApplicationException; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; + +import java.net.URI; +import java.util.Iterator; +import java.util.List; + +import static com.facebook.presto.server.ResourceUtil.assertRequest; +import static com.facebook.presto.server.ResourceUtil.createClientSessionForRequest; +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.base.Strings.isNullOrEmpty; +import static com.google.common.collect.Iterators.concat; +import static com.google.common.collect.Iterators.transform; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.MINUTES; +import static javax.ws.rs.core.Response.Status.INTERNAL_SERVER_ERROR; +import static javax.ws.rs.core.Response.status; + +@Path("/v1/execute") +public class ExecuteResource +{ + private final HttpServerInfo serverInfo; + private final HttpClient httpClient; + private final JsonCodec queryResultsCodec; + + @Inject + public ExecuteResource( + HttpServerInfo serverInfo, + @ForExecute HttpClient httpClient, + JsonCodec queryResultsCodec) + { + this.serverInfo = requireNonNull(serverInfo, "serverInfo is null"); + this.httpClient = requireNonNull(httpClient, "httpClient is null"); + this.queryResultsCodec = requireNonNull(queryResultsCodec, "queryResultsCodec is null"); + } + + @POST + @Produces(MediaType.APPLICATION_JSON) + public Response createQuery(String query, @Context HttpServletRequest servletRequest) + { + assertRequest(!isNullOrEmpty(query), "SQL query is empty"); + + ClientSession clientSession = createClientSessionForRequest(servletRequest, serverUri(), new Duration(2, MINUTES)); + StatementClient client = new StatementClient(httpClient, queryResultsCodec, clientSession, query); + + List columns = getColumns(client); + Iterator> iterator = flatten(new ResultsPageIterator(client)); + SimpleQueryResults results = new SimpleQueryResults(columns, iterator); + + return Response.ok(results, MediaType.APPLICATION_JSON_TYPE).build(); + } + + private URI serverUri() + { + checkState(serverInfo.getHttpUri() != null, "No HTTP URI for this server (HTTP disabled?)"); + return serverInfo.getHttpUri(); + } + + private static List getColumns(StatementClient client) + { + while (client.isValid()) { + List columns = client.current().getColumns(); + if (columns != null) { + return columns; + } + client.advance(); + } + + if (!client.isFailed()) { + throw internalServerError("No columns"); + } + throw internalServerError(failureMessage(client.finalResults())); + } + + @SuppressWarnings("RedundantTypeArguments") + private static Iterator flatten(Iterator> iterator) + { + // the explicit type argument is required by the Eclipse compiler + return concat(transform(iterator, Iterable::iterator)); + } + + private static class ResultsPageIterator + extends AbstractIterator>> + { + private final StatementClient client; + + private ResultsPageIterator(StatementClient client) + { + this.client = requireNonNull(client, "client is null"); + } + + @Override + protected Iterable> computeNext() + { + while (client.isValid()) { + Iterable> data = client.current().getData(); + client.advance(); + if (data != null) { + return data; + } + } + + if (client.isFailed()) { + throw internalServerError(failureMessage(client.finalResults())); + } + + return endOfData(); + } + } + + private static WebApplicationException internalServerError(String message) + { + return new WebApplicationException(status(INTERNAL_SERVER_ERROR).entity(message).build()); + } + + private static String failureMessage(QueryResults results) + { + return format("Query failed (#%s): %s", results.getId(), results.getError().getMessage()); + } + + public static class SimpleQueryResults + { + private final List columns; + private final Iterator> data; + + public SimpleQueryResults(List columns, Iterator> data) + { + this.columns = requireNonNull(columns, "columns is null"); + this.data = requireNonNull(data, "data is null"); + } + + @JsonProperty + public List getColumns() + { + return columns; + } + + @JsonProperty + public Iterator> getData() + { + return data; + } + } +} diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/cassandra/TestConstants.java b/presto-main/src/main/java/com/facebook/presto/server/ForExecute.java similarity index 54% rename from presto-product-tests/src/main/java/com/facebook/presto/tests/cassandra/TestConstants.java rename to presto-main/src/main/java/com/facebook/presto/server/ForExecute.java index 61830693345b..802126671380 100644 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/cassandra/TestConstants.java +++ b/presto-main/src/main/java/com/facebook/presto/server/ForExecute.java @@ -11,12 +11,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.facebook.presto.tests.cassandra; +package com.facebook.presto.server; -public final class TestConstants -{ - public static final String CONNECTOR_NAME = "cassandra"; - public static final String KEY_SPACE = "test"; +import javax.inject.Qualifier; - private TestConstants() {} -} +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.FIELD; +import static java.lang.annotation.ElementType.METHOD; +import static java.lang.annotation.ElementType.PARAMETER; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +@Retention(RUNTIME) +@Target({FIELD, PARAMETER, METHOD}) +@Qualifier +public @interface ForExecute +{} diff --git a/presto-main/src/main/java/com/facebook/presto/server/PluginManager.java b/presto-main/src/main/java/com/facebook/presto/server/PluginManager.java index dd9b6830b58b..1c36d89a4f23 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/PluginManager.java +++ b/presto-main/src/main/java/com/facebook/presto/server/PluginManager.java @@ -31,6 +31,7 @@ import com.facebook.presto.type.TypeRegistry; import com.google.common.collect.ImmutableList; import com.google.common.collect.Ordering; +import io.airlift.configuration.ConfigurationFactory; import io.airlift.http.server.HttpServerInfo; import io.airlift.log.Logger; import io.airlift.node.NodeInfo; @@ -90,6 +91,7 @@ public PluginManager( HttpServerInfo httpServerInfo, PluginManagerConfig config, ConnectorManager connectorManager, + ConfigurationFactory configurationFactory, Metadata metadata, ResourceGroupManager resourceGroupManager, AccessControlManager accessControlManager, @@ -100,6 +102,7 @@ public PluginManager( requireNonNull(nodeInfo, "nodeInfo is null"); requireNonNull(httpServerInfo, "httpServerInfo is null"); requireNonNull(config, "config is null"); + requireNonNull(configurationFactory, "configurationFactory is null"); installedPluginsDir = config.getInstalledPluginsDir(); if (config.getPlugins() == null) { @@ -119,6 +122,11 @@ public PluginManager( this.typeRegistry = requireNonNull(typeRegistry, "typeRegistry is null"); } + public boolean arePluginsLoaded() + { + return pluginsLoaded.get(); + } + public void loadPlugins() throws Exception { diff --git a/presto-main/src/main/java/com/facebook/presto/server/QueryResource.java b/presto-main/src/main/java/com/facebook/presto/server/QueryResource.java index 53513089ba1e..1da2910443ca 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/QueryResource.java +++ b/presto-main/src/main/java/com/facebook/presto/server/QueryResource.java @@ -13,23 +13,39 @@ */ package com.facebook.presto.server; +import com.facebook.presto.Session; +import com.facebook.presto.execution.QueryIdGenerator; import com.facebook.presto.execution.QueryInfo; import com.facebook.presto.execution.QueryManager; import com.facebook.presto.execution.StageId; +import com.facebook.presto.metadata.SessionPropertyManager; +import com.facebook.presto.security.AccessControl; import com.facebook.presto.spi.QueryId; +import com.facebook.presto.transaction.TransactionManager; import com.google.common.collect.ImmutableList; import javax.inject.Inject; +import javax.servlet.http.HttpServletRequest; import javax.ws.rs.DELETE; import javax.ws.rs.GET; +import javax.ws.rs.POST; import javax.ws.rs.Path; import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import javax.ws.rs.core.Response.Status; +import javax.ws.rs.core.UriInfo; +import java.net.URI; import java.util.List; import java.util.NoSuchElementException; +import static com.facebook.presto.server.ResourceUtil.assertRequest; +import static com.facebook.presto.server.ResourceUtil.createSessionForRequest; +import static com.google.common.base.Strings.isNullOrEmpty; +import static io.airlift.http.client.HttpUriBuilder.uriBuilderFrom; import static java.util.Objects.requireNonNull; /** @@ -39,11 +55,24 @@ public class QueryResource { private final QueryManager queryManager; + private final TransactionManager transactionManager; + private final AccessControl accessControl; + private final SessionPropertyManager sessionPropertyManager; + private final QueryIdGenerator queryIdGenerator; @Inject - public QueryResource(QueryManager queryManager) + public QueryResource( + QueryManager queryManager, + TransactionManager transactionManager, + AccessControl accessControl, + SessionPropertyManager sessionPropertyManager, + QueryIdGenerator queryIdGenerator) { this.queryManager = requireNonNull(queryManager, "queryManager is null"); + this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); + this.accessControl = requireNonNull(accessControl, "accessControl is null"); + this.sessionPropertyManager = requireNonNull(sessionPropertyManager, "sessionPropertyManager is null"); + this.queryIdGenerator = requireNonNull(queryIdGenerator, "queryIdGenerator is null"); } @GET @@ -76,6 +105,22 @@ public Response getQueryInfo(@PathParam("queryId") QueryId queryId) } } + @POST + @Produces(MediaType.APPLICATION_JSON) + public Response createQuery( + String statement, + @Context HttpServletRequest servletRequest, + @Context UriInfo uriInfo) + { + assertRequest(!isNullOrEmpty(statement), "SQL statement is empty"); + + Session session = createSessionForRequest(servletRequest, transactionManager, accessControl, sessionPropertyManager, queryIdGenerator.createNextQueryId()); + + QueryInfo queryInfo = queryManager.createQuery(session, statement); + URI pagesUri = uriBuilderFrom(uriInfo.getRequestUri()).appendPath(queryInfo.getQueryId().toString()).build(); + return Response.created(pagesUri).entity(queryInfo).build(); + } + @DELETE @Path("{queryId}") public void cancelQuery(@PathParam("queryId") QueryId queryId) diff --git a/presto-main/src/main/java/com/facebook/presto/server/ResourceUtil.java b/presto-main/src/main/java/com/facebook/presto/server/ResourceUtil.java index edd37c15f46b..07e584183b1b 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/ResourceUtil.java +++ b/presto-main/src/main/java/com/facebook/presto/server/ResourceUtil.java @@ -15,6 +15,7 @@ import com.facebook.presto.Session; import com.facebook.presto.Session.SessionBuilder; +import com.facebook.presto.client.ClientSession; import com.facebook.presto.metadata.SessionPropertyManager; import com.facebook.presto.security.AccessControl; import com.facebook.presto.spi.QueryId; @@ -27,6 +28,7 @@ import com.facebook.presto.transaction.TransactionId; import com.facebook.presto.transaction.TransactionManager; import com.google.common.base.Splitter; +import io.airlift.units.Duration; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.WebApplicationException; @@ -35,6 +37,7 @@ import javax.ws.rs.core.Response.Status; import java.io.UnsupportedEncodingException; +import java.net.URI; import java.net.URLDecoder; import java.net.URLEncoder; import java.security.Principal; @@ -62,6 +65,7 @@ import static com.google.common.base.Strings.nullToEmpty; import static com.google.common.net.HttpHeaders.USER_AGENT; import static java.lang.String.format; +import static java.util.Objects.requireNonNull; final class ResourceUtil { @@ -88,11 +92,7 @@ public static Session createSessionForRequest( accessControl.checkCanSetUser(principal, user); } catch (AccessDeniedException e) { - throw new WebApplicationException( - e, - Response.status(Status.FORBIDDEN) - .entity("Access denied: " + e.getMessage()) - .build()); + throw new WebApplicationException(e.getMessage(), Status.FORBIDDEN); } Identity identity = new Identity(user, Optional.ofNullable(principal)); @@ -161,6 +161,54 @@ else if (nameParts.size() == 2) { return session; } + public static ClientSession createClientSessionForRequest(HttpServletRequest request, URI server, Duration clientRequestTimeout) + { + requireNonNull(request, "request is null"); + requireNonNull(server, "server is null"); + requireNonNull(clientRequestTimeout, "clientRequestTimeout is null"); + + String catalog = trimEmptyToNull(request.getHeader(PRESTO_CATALOG)); + String schema = trimEmptyToNull(request.getHeader(PRESTO_SCHEMA)); + assertRequest((catalog != null) || (schema == null), "Schema is set but catalog is not"); + + String user = trimEmptyToNull(request.getHeader(PRESTO_USER)); + assertRequest(user != null, "User must be set"); + + Principal principal = request.getUserPrincipal(); + + String source = request.getHeader(PRESTO_SOURCE); + + Identity identity = new Identity(user, Optional.ofNullable(principal)); + + String transactionId = trimEmptyToNull(request.getHeader(PRESTO_TRANSACTION_ID)); + + String timeZoneId = request.getHeader(PRESTO_TIME_ZONE); + + String language = request.getHeader(PRESTO_LANGUAGE); + Locale locale = null; + if (language != null) { + locale = Locale.forLanguageTag(language); + } + + Map sessionProperties = parseSessionHeaders(request); + + Map preparedStatements = parsePreparedStatementsHeaders(request); + + return new ClientSession( + server, + identity.getUser(), + source, + catalog, + schema, + timeZoneId, + locale, + sessionProperties, + preparedStatements, + transactionId, + false, + clientRequestTimeout); + } + private static List splitSessionHeader(Enumeration headers) { Splitter splitter = Splitter.on(',').trimResults().omitEmptyStrings(); diff --git a/presto-main/src/main/java/com/facebook/presto/server/ServerMainModule.java b/presto-main/src/main/java/com/facebook/presto/server/ServerMainModule.java index b2987979c68c..8d60e422b877 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/ServerMainModule.java +++ b/presto-main/src/main/java/com/facebook/presto/server/ServerMainModule.java @@ -398,12 +398,11 @@ protected void setup(Binder binder) // Spiller binder.bind(SpillerFactory.class).to(BinarySpillerFactory.class).in(Scopes.SINGLETON); - newExporter(binder).export(SpillerFactory.class).withGeneratedName(); } @Provides @Singleton - public static ServerInfo createServerInfo(NodeVersion nodeVersion, NodeInfo nodeInfo, ServerConfig serverConfig) + public ServerInfo createServerInfo(NodeVersion nodeVersion, NodeInfo nodeInfo, ServerConfig serverConfig) { return new ServerInfo(nodeVersion, nodeInfo.getEnvironment(), serverConfig.isCoordinator()); } @@ -411,7 +410,7 @@ public static ServerInfo createServerInfo(NodeVersion nodeVersion, NodeInfo node @Provides @Singleton @ForExchange - public static ScheduledExecutorService createExchangeExecutor(ExchangeClientConfig config) + public ScheduledExecutorService createExchangeExecutor(ExchangeClientConfig config) { return newScheduledThreadPool(config.getClientThreads(), daemonThreadsNamed("exchange-client-%s")); } diff --git a/presto-main/src/main/java/com/facebook/presto/server/StatementResource.java b/presto-main/src/main/java/com/facebook/presto/server/StatementResource.java index 42a95779f7f9..a0cf0a13eaca 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/StatementResource.java +++ b/presto-main/src/main/java/com/facebook/presto/server/StatementResource.java @@ -30,6 +30,7 @@ import com.facebook.presto.execution.StageInfo; import com.facebook.presto.execution.StageState; import com.facebook.presto.execution.TaskInfo; +import com.facebook.presto.execution.buffer.BufferInfo; import com.facebook.presto.execution.buffer.OutputBufferInfo; import com.facebook.presto.metadata.SessionPropertyManager; import com.facebook.presto.operator.ExchangeClient; @@ -45,6 +46,7 @@ import com.facebook.presto.spi.type.TypeSignature; import com.facebook.presto.transaction.TransactionId; import com.facebook.presto.transaction.TransactionManager; +import com.google.common.base.Preconditions; import com.google.common.collect.AbstractIterator; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -528,11 +530,17 @@ private synchronized void updateExchangeClient(StageInfo outputStage) if (!outputStage.getState().isDone()) { for (TaskInfo taskInfo : outputStage.getTasks()) { OutputBufferInfo outputBuffers = taskInfo.getOutputBuffers(); - if (outputBuffers.getState().canAddBuffers()) { - // output buffer are still being created + List buffers = outputBuffers.getBuffers(); + if (buffers.isEmpty() || outputBuffers.getState().canAddBuffers()) { + // output buffer has not been created yet continue; } - OutputBufferId bufferId = new OutputBufferId(0); + Preconditions.checkState(buffers.size() == 1, + "Expected a single output buffer for task %s, but found %s", + taskInfo.getTaskStatus().getTaskId(), + buffers); + + OutputBufferId bufferId = Iterables.getOnlyElement(buffers).getBufferId(); URI uri = uriBuilderFrom(taskInfo.getTaskStatus().getSelf()).appendPath("results").appendPath(bufferId.toString()).build(); exchangeClient.addLocation(uri); } diff --git a/presto-main/src/main/java/com/facebook/presto/server/TaskResource.java b/presto-main/src/main/java/com/facebook/presto/server/TaskResource.java index 8dbb9b47caf6..8333647e0d41 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/TaskResource.java +++ b/presto-main/src/main/java/com/facebook/presto/server/TaskResource.java @@ -210,19 +210,11 @@ public TaskInfo deleteTask(@PathParam("taskId") TaskId taskId, @Context UriInfo uriInfo) { requireNonNull(taskId, "taskId is null"); - TaskInfo taskInfo; if (abort) { - taskInfo = taskManager.abortTask(taskId); + return taskManager.abortTask(taskId); } - else { - taskInfo = taskManager.cancelTask(taskId); - } - - if (shouldSummarize(uriInfo)) { - taskInfo = taskInfo.summarize(); - } - return taskInfo; + return taskManager.cancelTask(taskId); } @GET diff --git a/presto-main/src/main/java/com/facebook/presto/server/ThreadResource.java b/presto-main/src/main/java/com/facebook/presto/server/ThreadResource.java index 4d96618c6639..68177eaf72f1 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/ThreadResource.java +++ b/presto-main/src/main/java/com/facebook/presto/server/ThreadResource.java @@ -50,7 +50,7 @@ public String getUi() @GET @Path("/v1/thread") @Produces(MediaType.APPLICATION_JSON) - public static List getThreadInfo() + public List getThreadInfo() { ThreadMXBean mbean = ManagementFactory.getThreadMXBean(); diff --git a/presto-main/src/main/java/com/facebook/presto/server/remotetask/ContinuousTaskStatusFetcher.java b/presto-main/src/main/java/com/facebook/presto/server/remotetask/ContinuousTaskStatusFetcher.java index d769462a5612..976e9f7a1d67 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/remotetask/ContinuousTaskStatusFetcher.java +++ b/presto-main/src/main/java/com/facebook/presto/server/remotetask/ContinuousTaskStatusFetcher.java @@ -233,9 +233,6 @@ void updateTaskStatus(TaskStatus newValue) }); if (taskMismatch.get()) { - // This will also set the task status to FAILED state directly. - // Additionally, this will issue a DELETE for the task to the worker. - // While sending the DELETE is not required, it is preferred because a task was created by the previous request. onFail.accept(new PrestoException(REMOTE_TASK_MISMATCH, format("%s (%s)", REMOTE_TASK_MISMATCH_ERROR, HostAddress.fromUri(getTaskStatus().getSelf())))); } } diff --git a/presto-main/src/main/java/com/facebook/presto/server/remotetask/HttpRemoteTask.java b/presto-main/src/main/java/com/facebook/presto/server/remotetask/HttpRemoteTask.java index 9cc6730f2c07..45a84a64e750 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/remotetask/HttpRemoteTask.java +++ b/presto-main/src/main/java/com/facebook/presto/server/remotetask/HttpRemoteTask.java @@ -17,7 +17,6 @@ import com.facebook.presto.ScheduledSplit; import com.facebook.presto.Session; import com.facebook.presto.TaskSource; -import com.facebook.presto.execution.FutureStateChange; import com.facebook.presto.execution.NodeTaskMap.PartitionedSplitCountTracker; import com.facebook.presto.execution.RemoteTask; import com.facebook.presto.execution.StateMachine.StateChangeListener; @@ -84,6 +83,7 @@ import static com.google.common.base.MoreObjects.toStringHelper; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; +import static io.airlift.concurrent.MoreFutures.unmodifiableFuture; import static io.airlift.http.client.FullJsonResponseHandler.createFullJsonResponseHandler; import static io.airlift.http.client.HttpUriBuilder.uriBuilderFrom; import static io.airlift.http.client.JsonBodyGenerator.jsonBodyGenerator; @@ -126,9 +126,10 @@ public final class HttpRemoteTask private final Set noMoreSplits = new HashSet<>(); @GuardedBy("this") private final AtomicReference outputBuffers = new AtomicReference<>(); - private final FutureStateChange whenSplitQueueHasSpace = new FutureStateChange<>(); @GuardedBy("this") - private boolean splitQueueHasSpace = true; + private CompletableFuture whenSplitQueueHasSpace = completedFuture(null); + @GuardedBy("this") + private CompletableFuture unmodifiableWhenSplitQueueHasSpace = completedFuture(null); @GuardedBy("this") private OptionalInt whenSplitQueueHasSpaceThreshold = OptionalInt.empty(); @@ -149,8 +150,6 @@ public final class HttpRemoteTask private final PartitionedSplitCountTracker partitionedSplitCountTracker; - private final AtomicBoolean aborting = new AtomicBoolean(false); - public HttpRemoteTask(Session session, TaskId taskId, String nodeId, @@ -395,10 +394,7 @@ public synchronized CompletableFuture whenSplitQueueHasSpace(int threshold) whenSplitQueueHasSpaceThreshold = OptionalInt.of(threshold); updateSplitQueueSpace(); } - if (splitQueueHasSpace) { - return completedFuture(null); - } - return whenSplitQueueHasSpace.createNewListener(); + return unmodifiableWhenSplitQueueHasSpace; } private synchronized void updateSplitQueueSpace() @@ -406,10 +402,32 @@ private synchronized void updateSplitQueueSpace() if (!whenSplitQueueHasSpaceThreshold.isPresent()) { return; } - splitQueueHasSpace = getQueuedPartitionedSplitCount() < whenSplitQueueHasSpaceThreshold.getAsInt(); - if (splitQueueHasSpace) { - whenSplitQueueHasSpace.complete(null, executor); + if (getQueuedPartitionedSplitCount() < whenSplitQueueHasSpaceThreshold.getAsInt()) { + if (!whenSplitQueueHasSpace.isDone()) { + fireSplitQueueHasSpace(whenSplitQueueHasSpace); + whenSplitQueueHasSpace = completedFuture(null); + unmodifiableWhenSplitQueueHasSpace = unmodifiableFuture(whenSplitQueueHasSpace); + } } + else { + if (whenSplitQueueHasSpace.isDone()) { + whenSplitQueueHasSpace = new CompletableFuture<>(); + unmodifiableWhenSplitQueueHasSpace = unmodifiableFuture(whenSplitQueueHasSpace); + } + } + } + + private void fireSplitQueueHasSpace(CompletableFuture future) + { + executor.execute(() -> { + checkState(!Thread.holdsLock(this), "Can not notify split queue future while holding the lock"); + try { + future.complete(null); + } + catch (Throwable e) { + log.error(e, "Error notifying split queue future for %s", taskId); + } + }); } private synchronized void processTaskUpdate(TaskInfo newValue, List sources) @@ -551,8 +569,11 @@ private synchronized void cleanUpTask() pendingSplits.clear(); pendingSourceSplitCount = 0; partitionedSplitCountTracker.setPartitionedSplitCount(getPartitionedSplitCount()); - splitQueueHasSpace = true; - whenSplitQueueHasSpace.complete(null, executor); + if (!whenSplitQueueHasSpace.isDone()) { + fireSplitQueueHasSpace(whenSplitQueueHasSpace); + whenSplitQueueHasSpace = completedFuture(null); + unmodifiableWhenSplitQueueHasSpace = unmodifiableFuture(whenSplitQueueHasSpace); + } // cancel pending request if (currentRequest != null) { @@ -600,16 +621,6 @@ private synchronized void abort(TaskStatus status) } private void scheduleAsyncCleanupRequest(Backoff cleanupBackoff, Request request, String action) - { - if (!aborting.compareAndSet(false, true)) { - // Do not initiate another round of cleanup requests if one had been initiated. - // Otherwise, we can get into an asynchronous recursion here. For example, when aborting a task after REMOTE_TASK_MISMATCH. - return; - } - doScheduleAsyncCleanupRequest(cleanupBackoff, request, action); - } - - private void doScheduleAsyncCleanupRequest(Backoff cleanupBackoff, Request request, String action) { Futures.addCallback(httpClient.executeAsync(request, createFullJsonResponseHandler(taskInfoCodec)), new FutureCallback>() { @@ -643,10 +654,10 @@ public void onFailure(Throwable t) // reschedule long delayNanos = cleanupBackoff.getBackoffDelayNanos(); if (delayNanos == 0) { - doScheduleAsyncCleanupRequest(cleanupBackoff, request, action); + scheduleAsyncCleanupRequest(cleanupBackoff, request, action); } else { - errorScheduledExecutor.schedule(() -> doScheduleAsyncCleanupRequest(cleanupBackoff, request, action), delayNanos, NANOSECONDS); + errorScheduledExecutor.schedule(() -> scheduleAsyncCleanupRequest(cleanupBackoff, request, action), delayNanos, NANOSECONDS); } } }, executor); diff --git a/presto-main/src/main/java/com/facebook/presto/server/security/SpnegoFilter.java b/presto-main/src/main/java/com/facebook/presto/server/security/SpnegoFilter.java index 3e9ee51d632a..720ec4576e5c 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/security/SpnegoFilter.java +++ b/presto-main/src/main/java/com/facebook/presto/server/security/SpnegoFilter.java @@ -43,7 +43,6 @@ import javax.servlet.http.HttpServletResponse; import java.io.IOException; -import java.io.InputStream; import java.net.InetAddress; import java.net.UnknownHostException; import java.security.Principal; @@ -54,8 +53,6 @@ import java.util.Map; import java.util.Optional; -import static com.google.common.io.ByteStreams.copy; -import static com.google.common.io.ByteStreams.nullOutputStream; import static java.lang.String.format; import static javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag.REQUIRED; import static javax.servlet.http.HttpServletResponse.SC_UNAUTHORIZED; @@ -177,7 +174,7 @@ public Principal getUserPrincipal() } } - sendChallenge(request, response, includeRealm, requestSpnegoToken); + sendChallenge(response, includeRealm, requestSpnegoToken); } private Optional authenticate(String token) @@ -214,17 +211,9 @@ private Optional authenticate(String token) return Optional.empty(); } - private static void sendChallenge(HttpServletRequest request, HttpServletResponse response, boolean includeRealm, String invalidSpnegoToken) + private static void sendChallenge(HttpServletResponse response, boolean includeRealm, String invalidSpnegoToken) throws IOException { - // If we send the challenge without consuming the body of the request, - // the Jetty server will close the connection after sending the response. - // The client interprets this as a failed request and does not resend - // the request with the authentication header. - // We can avoid this behavior in the Jetty client by reading and discarding - // the entire body of the unauthenticated request before sending the response. - skipRequestBody(request); - if (invalidSpnegoToken != null) { response.sendError(SC_UNAUTHORIZED, format("Authentication failed for token %s", invalidSpnegoToken)); } @@ -234,14 +223,6 @@ private static void sendChallenge(HttpServletRequest request, HttpServletRespons response.setHeader(HttpHeaders.WWW_AUTHENTICATE, formatAuthenticationHeader(includeRealm, Optional.empty())); } - private static void skipRequestBody(HttpServletRequest request) - throws IOException - { - try (InputStream inputStream = request.getInputStream()) { - copy(inputStream, nullOutputStream()); - } - } - private static String formatAuthenticationHeader(boolean includeRealm, Optional token) { StringBuilder header = new StringBuilder(NEGOTIATE_SCHEME); diff --git a/presto-main/src/main/java/com/facebook/presto/server/testing/TestingPrestoServer.java b/presto-main/src/main/java/com/facebook/presto/server/testing/TestingPrestoServer.java index 11c499610f26..e5cedf20fc36 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/testing/TestingPrestoServer.java +++ b/presto-main/src/main/java/com/facebook/presto/server/testing/TestingPrestoServer.java @@ -145,13 +145,13 @@ public synchronized boolean isWorkerShutdown() public TestingPrestoServer() throws Exception { - this(ImmutableList.of()); + this(ImmutableList.of()); } public TestingPrestoServer(List additionalModules) throws Exception { - this(true, ImmutableMap.of(), null, null, new SqlParserOptions(), additionalModules); + this(true, ImmutableMap.of(), null, null, new SqlParserOptions(), additionalModules); } public TestingPrestoServer(boolean coordinator, diff --git a/presto-main/src/main/java/com/facebook/presto/spiller/BinaryFileSpiller.java b/presto-main/src/main/java/com/facebook/presto/spiller/BinaryFileSpiller.java index 38dc2895ba59..594c73d3065a 100644 --- a/presto-main/src/main/java/com/facebook/presto/spiller/BinaryFileSpiller.java +++ b/presto-main/src/main/java/com/facebook/presto/spiller/BinaryFileSpiller.java @@ -25,8 +25,6 @@ import io.airlift.slice.RuntimeIOException; import io.airlift.slice.SliceOutput; -import javax.annotation.concurrent.NotThreadSafe; - import java.io.BufferedInputStream; import java.io.BufferedOutputStream; import java.io.FileInputStream; @@ -39,39 +37,28 @@ import java.util.Iterator; import java.util.List; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicLong; import java.util.stream.IntStream; import java.util.stream.Stream; import static com.facebook.presto.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; import static com.facebook.presto.util.ImmutableCollectors.toImmutableList; -import static com.google.common.base.Preconditions.checkState; import static java.util.Objects.requireNonNull; import static java.util.stream.Collectors.toList; -@NotThreadSafe public class BinaryFileSpiller implements Spiller { private final Path targetDirectory; private final Closer closer = Closer.create(); private final BlockEncodingSerde blockEncodingSerde; - private final AtomicLong spilledDataSize; - - private final ListeningExecutorService executor; private int spillsCount; - private CompletableFuture previousSpill = CompletableFuture.completedFuture(null); + private final ListeningExecutorService executor; - public BinaryFileSpiller( - BlockEncodingSerde blockEncodingSerde, - ListeningExecutorService executor, - Path spillPath, - AtomicLong spilledDataSize) + public BinaryFileSpiller(BlockEncodingSerde blockEncodingSerde, ListeningExecutorService executor, Path spillPath) { this.blockEncodingSerde = requireNonNull(blockEncodingSerde, "blockEncodingSerde is null"); this.executor = requireNonNull(executor, "executor is null"); - this.spilledDataSize = requireNonNull(spilledDataSize, "spilledDataSize is null"); try { this.targetDirectory = Files.createTempDirectory(spillPath, "presto-spill"); } @@ -83,18 +70,17 @@ public BinaryFileSpiller( @Override public CompletableFuture spill(Iterator pageIterator) { - checkState(previousSpill.isDone()); Path spillPath = getPath(spillsCount++); - previousSpill = MoreFutures.toCompletableFuture(executor.submit( - () -> writePages(pageIterator, spillPath))); - return previousSpill; + return MoreFutures.toCompletableFuture(executor.submit( + () -> writePages(pageIterator, spillPath) + )); } private void writePages(Iterator pageIterator, Path spillPath) { try (SliceOutput output = new OutputStreamSliceOutput(new BufferedOutputStream(new FileOutputStream(spillPath.toFile())))) { - spilledDataSize.addAndGet(PagesSerde.writePages(blockEncodingSerde, output, pageIterator)); + PagesSerde.writePages(blockEncodingSerde, output, pageIterator); } catch (RuntimeIOException | IOException e) { throw new PrestoException(GENERIC_INTERNAL_ERROR, "Failed to spill pages", e); @@ -104,7 +90,6 @@ private void writePages(Iterator pageIterator, Path spillPath) @Override public List> getSpills() { - checkState(previousSpill.isDone()); return IntStream.range(0, spillsCount) .mapToObj(i -> readPages(getPath(i))) .collect(toImmutableList()); diff --git a/presto-main/src/main/java/com/facebook/presto/spiller/BinarySpillerFactory.java b/presto-main/src/main/java/com/facebook/presto/spiller/BinarySpillerFactory.java index b2001920e95c..984257955e16 100644 --- a/presto-main/src/main/java/com/facebook/presto/spiller/BinarySpillerFactory.java +++ b/presto-main/src/main/java/com/facebook/presto/spiller/BinarySpillerFactory.java @@ -31,7 +31,7 @@ import static java.util.concurrent.Executors.newFixedThreadPool; public class BinarySpillerFactory - extends SpillerFactoryWithStats + implements SpillerFactory { public static final String SPILLER_THREAD_NAME_PREFIX = "binary-spiller"; @@ -65,6 +65,6 @@ private static ListeningExecutorService createExecutorServiceOfSize(int nThreads @Override public Spiller create(List types) { - return new BinaryFileSpiller(blockEncodingSerde, executor, spillPath, spilledBytes); + return new BinaryFileSpiller(blockEncodingSerde, executor, spillPath); } } diff --git a/presto-main/src/main/java/com/facebook/presto/spiller/SpillerFactory.java b/presto-main/src/main/java/com/facebook/presto/spiller/SpillerFactory.java index 43254a098cbe..3e1f37d27d75 100644 --- a/presto-main/src/main/java/com/facebook/presto/spiller/SpillerFactory.java +++ b/presto-main/src/main/java/com/facebook/presto/spiller/SpillerFactory.java @@ -15,14 +15,10 @@ package com.facebook.presto.spiller; import com.facebook.presto.spi.type.Type; -import org.weakref.jmx.Managed; import java.util.List; public interface SpillerFactory { Spiller create(List types); - - @Managed - long getSpilledBytes(); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/ExpressionUtils.java b/presto-main/src/main/java/com/facebook/presto/sql/ExpressionUtils.java index b1e658f24532..35a263fa6046 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/ExpressionUtils.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/ExpressionUtils.java @@ -21,7 +21,6 @@ import com.facebook.presto.sql.tree.ExpressionRewriter; import com.facebook.presto.sql.tree.ExpressionTreeRewriter; import com.facebook.presto.sql.tree.IsNullPredicate; -import com.facebook.presto.sql.tree.LambdaExpression; import com.facebook.presto.sql.tree.LogicalBinaryExpression; import com.facebook.presto.sql.tree.NotExpression; import com.facebook.presto.sql.tree.QualifiedNameReference; @@ -263,12 +262,6 @@ public Expression rewriteQualifiedNameReference(QualifiedNameReference node, Voi { return new SymbolReference(node.getName().toString()); } - - @Override - public Expression rewriteLambdaExpression(LambdaExpression node, Void context, ExpressionTreeRewriter treeRewriter) - { - return new LambdaExpression(node.getArguments(), treeRewriter.rewrite(node.getBody(), context)); - } }, expression); } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/AggregationAnalyzer.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/AggregationAnalyzer.java index 510424e04878..71bc5c27a0f1 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/AggregationAnalyzer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/AggregationAnalyzer.java @@ -37,7 +37,6 @@ import com.facebook.presto.sql.tree.InPredicate; import com.facebook.presto.sql.tree.IsNotNullPredicate; import com.facebook.presto.sql.tree.IsNullPredicate; -import com.facebook.presto.sql.tree.LambdaExpression; import com.facebook.presto.sql.tree.LikePredicate; import com.facebook.presto.sql.tree.Literal; import com.facebook.presto.sql.tree.LogicalBinaryExpression; @@ -89,11 +88,10 @@ class AggregationAnalyzer private final Metadata metadata; private final Set columnReferences; private final List parameters; - private final boolean isDescribe; private final Scope scope; - public AggregationAnalyzer(List groupByExpressions, Metadata metadata, Scope scope, Set columnReferences, List parameters, boolean isDescribe) + public AggregationAnalyzer(List groupByExpressions, Metadata metadata, Scope scope, Set columnReferences, List parameters) { requireNonNull(groupByExpressions, "groupByExpressions is null"); requireNonNull(metadata, "metadata is null"); @@ -105,7 +103,6 @@ public AggregationAnalyzer(List groupByExpressions, Metadata metadat this.metadata = metadata; this.columnReferences = ImmutableSet.copyOf(columnReferences); this.parameters = parameters; - this.isDescribe = isDescribe; this.expressions = groupByExpressions.stream() .map(e -> ExpressionTreeRewriter.rewriteWith(new ParameterRewriter(parameters), e)) .collect(toImmutableList()); @@ -318,13 +315,6 @@ protected Boolean visitFunctionCall(FunctionCall node, Void context) return node.getArguments().stream().allMatch(expression -> process(expression, context)); } - @Override - protected Boolean visitLambdaExpression(LambdaExpression node, Void context) - { - // Lambda does not support capture yet - return true; - } - @Override public Boolean visitWindow(Window node, Void context) { @@ -503,9 +493,6 @@ public Boolean visitRow(Row node, final Void context) @Override public Boolean visitParameter(Parameter node, Void context) { - if (isDescribe) { - return true; - } checkArgument(node.getPosition() < parameters.size(), "Invalid parameter number %s, max values is %s", node.getPosition(), parameters.size() - 1); return process(parameters.get(node.getPosition()), context); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/Analysis.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/Analysis.java index aff564a58639..b8c30cf52b3e 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/Analysis.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/Analysis.java @@ -24,9 +24,7 @@ import com.facebook.presto.sql.tree.FunctionCall; import com.facebook.presto.sql.tree.InPredicate; import com.facebook.presto.sql.tree.Join; -import com.facebook.presto.sql.tree.LambdaArgumentDeclaration; import com.facebook.presto.sql.tree.Node; -import com.facebook.presto.sql.tree.QualifiedNameReference; import com.facebook.presto.sql.tree.QuantifiedComparisonExpression; import com.facebook.presto.sql.tree.Query; import com.facebook.presto.sql.tree.QuerySpecification; @@ -87,7 +85,6 @@ public class Analysis private final Set typeOnlyCoercions = newIdentityHashSet(); private final IdentityHashMap relationCoercions = new IdentityHashMap<>(); private final IdentityHashMap functionSignature = new IdentityHashMap<>(); - private final IdentityHashMap lambdaArgumentReferences = new IdentityHashMap<>(); private final IdentityHashMap columns = new IdentityHashMap<>(); @@ -198,21 +195,6 @@ public Type getCoercion(Expression expression) return coercions.get(expression); } - public void addLambdaArgumentReferences(IdentityHashMap lambdaArgumentReferences) - { - this.lambdaArgumentReferences.putAll(lambdaArgumentReferences); - } - - public LambdaArgumentDeclaration getLambdaArgumentReference(QualifiedNameReference qualifiedNameReference) - { - return lambdaArgumentReferences.get(qualifiedNameReference); - } - - public Map getLambdaArgumentReferences() - { - return lambdaArgumentReferences; - } - public void setGroupingSets(QuerySpecification node, List> expressions) { groupByExpressions.put(node, expressions); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionAnalysis.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionAnalysis.java index d6e4aab4d538..8fd638ab1174 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionAnalysis.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionAnalysis.java @@ -17,8 +17,6 @@ import com.facebook.presto.sql.tree.ExistsPredicate; import com.facebook.presto.sql.tree.Expression; import com.facebook.presto.sql.tree.InPredicate; -import com.facebook.presto.sql.tree.LambdaArgumentDeclaration; -import com.facebook.presto.sql.tree.QualifiedNameReference; import com.facebook.presto.sql.tree.QuantifiedComparisonExpression; import com.facebook.presto.sql.tree.SubqueryExpression; import com.google.common.collect.ImmutableSet; @@ -38,8 +36,6 @@ public class ExpressionAnalysis private final Set scalarSubqueries; private final Set existsSubqueries; private final Set quantifiedComparisons; - // For lambda argument references, maps each QualifiedNameReference to the referenced LambdaArgumentDeclaration - private final IdentityHashMap lambdaArgumentReferences; public ExpressionAnalysis( IdentityHashMap expressionTypes, @@ -49,8 +45,7 @@ public ExpressionAnalysis( Set existsSubqueries, Set columnReferences, Set typeOnlyCoercions, - Set quantifiedComparisons, - IdentityHashMap lambdaArgumentReferences) + Set quantifiedComparisons) { this.expressionTypes = requireNonNull(expressionTypes, "expressionTypes is null"); this.expressionCoercions = requireNonNull(expressionCoercions, "expressionCoercions is null"); @@ -60,7 +55,6 @@ public ExpressionAnalysis( this.scalarSubqueries = requireNonNull(scalarSubqueries, "subqueryInPredicates is null"); this.existsSubqueries = requireNonNull(existsSubqueries, "existsSubqueries is null"); this.quantifiedComparisons = requireNonNull(quantifiedComparisons, "quantifiedComparisons is null"); - this.lambdaArgumentReferences = requireNonNull(lambdaArgumentReferences, "lambdaArgumentReferences is null"); } public Type getType(Expression expression) @@ -78,11 +72,6 @@ public Type getCoercion(Expression expression) return expressionCoercions.get(expression); } - public LambdaArgumentDeclaration getLambdaArgumentReference(QualifiedNameReference qualifiedNameReference) - { - return lambdaArgumentReferences.get(qualifiedNameReference); - } - public boolean isTypeOnlyCoercion(Expression expression) { return typeOnlyCoercions.contains(expression); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionAnalyzer.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionAnalyzer.java index c0af422224e4..b7b3f1f3c581 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionAnalyzer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/ExpressionAnalyzer.java @@ -28,6 +28,7 @@ import com.facebook.presto.spi.type.Decimals; import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.TypeManager; +import com.facebook.presto.spi.type.TypeSignature; import com.facebook.presto.spi.type.TypeSignatureParameter; import com.facebook.presto.spi.type.VarcharType; import com.facebook.presto.sql.parser.SqlParser; @@ -59,8 +60,6 @@ import com.facebook.presto.sql.tree.IntervalLiteral; import com.facebook.presto.sql.tree.IsNotNullPredicate; import com.facebook.presto.sql.tree.IsNullPredicate; -import com.facebook.presto.sql.tree.LambdaArgumentDeclaration; -import com.facebook.presto.sql.tree.LambdaExpression; import com.facebook.presto.sql.tree.LikePredicate; import com.facebook.presto.sql.tree.LogicalBinaryExpression; import com.facebook.presto.sql.tree.LongLiteral; @@ -86,7 +85,6 @@ import com.facebook.presto.sql.tree.TryExpression; import com.facebook.presto.sql.tree.WhenClause; import com.facebook.presto.sql.tree.WindowFrame; -import com.facebook.presto.type.FunctionType; import com.facebook.presto.type.RowType; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -96,7 +94,6 @@ import javax.annotation.Nullable; import java.util.ArrayList; -import java.util.HashMap; import java.util.IdentityHashMap; import java.util.List; import java.util.Map; @@ -126,7 +123,6 @@ import static com.facebook.presto.sql.analyzer.SemanticErrorCode.INVALID_PARAMETER_USAGE; import static com.facebook.presto.sql.analyzer.SemanticErrorCode.MULTIPLE_FIELDS_FROM_SUBQUERY; import static com.facebook.presto.sql.analyzer.SemanticErrorCode.NOT_SUPPORTED; -import static com.facebook.presto.sql.analyzer.SemanticErrorCode.STANDALONE_LAMBDA; import static com.facebook.presto.sql.analyzer.SemanticErrorCode.TYPE_MISMATCH; import static com.facebook.presto.sql.analyzer.SemanticExceptions.throwMissingAttributeException; import static com.facebook.presto.sql.tree.Extract.Field.TIMEZONE_HOUR; @@ -165,8 +161,6 @@ public class ExpressionAnalyzer private final Set columnReferences = newIdentityHashSet(); private final IdentityHashMap expressionTypes = new IdentityHashMap<>(); private final Set quantifiedComparisons = newIdentityHashSet(); - // For lambda argument references, maps each QualifiedNameReference to the referenced LambdaArgumentDeclaration - private final IdentityHashMap lambdaArgumentReferences = new IdentityHashMap<>(); private final Session session; private final List parameters; @@ -219,21 +213,10 @@ public Set getColumnReferences() return ImmutableSet.copyOf(columnReferences); } - public IdentityHashMap getLambdaArgumentReferences() - { - return lambdaArgumentReferences; - } - public Type analyze(Expression expression, Scope scope) { - Visitor visitor = new Visitor(scope, symbolTypes); - return visitor.process(expression, new StackableAstVisitor.StackableAstVisitorContext<>(Context.notInLambda())); - } - - private Type analyze(Expression expression, Scope scope, Context context) - { - Visitor visitor = new Visitor(scope, symbolTypes); - return visitor.process(expression, new StackableAstVisitor.StackableAstVisitorContext<>(context)); + Visitor visitor = new Visitor(scope); + return visitor.process(expression, new StackableAstVisitor.StackableAstVisitorContext<>(null)); } public Set getScalarSubqueries() @@ -252,18 +235,18 @@ public Set getQuantifiedComparisons() } private class Visitor - extends StackableAstVisitor + extends StackableAstVisitor { private final Scope scope; - private Visitor(Scope scope, Map symbolTypes) + private Visitor(Scope scope) { this.scope = requireNonNull(scope, "scope is null"); } @SuppressWarnings("SuspiciousMethodCalls") @Override - public Type process(Node node, @Nullable StackableAstVisitorContext context) + public Type process(Node node, @Nullable StackableAstVisitorContext context) { // don't double process a node Type type = expressionTypes.get(node); @@ -274,7 +257,7 @@ public Type process(Node node, @Nullable StackableAstVisitorContext con } @Override - protected Type visitRow(Row node, StackableAstVisitorContext context) + protected Type visitRow(Row node, StackableAstVisitorContext context) { List types = node.getItems().stream() .map((child) -> process(child, context)) @@ -287,7 +270,7 @@ protected Type visitRow(Row node, StackableAstVisitorContext context) } @Override - protected Type visitCurrentTime(CurrentTime node, StackableAstVisitorContext context) + protected Type visitCurrentTime(CurrentTime node, StackableAstVisitorContext context) { if (node.getPrecision() != null) { throw new SemanticException(NOT_SUPPORTED, node, "non-default precision not yet supported"); @@ -319,16 +302,8 @@ protected Type visitCurrentTime(CurrentTime node, StackableAstVisitorContext context) + protected Type visitSymbolReference(SymbolReference node, StackableAstVisitorContext context) { - if (context.getContext().isInLambda()) { - LambdaArgumentDeclaration lambdaArgumentDeclaration = context.getContext().getNameToLambdaArgumentDeclarationMap().get(node.getName()); - if (lambdaArgumentDeclaration != null) { - Type result = expressionTypes.get(lambdaArgumentDeclaration); - expressionTypes.put(node, result); - return result; - } - } Type type = symbolTypes.get(Symbol.from(node)); checkArgument(type != null, "No type for symbol %s", node.getName()); expressionTypes.put(node, type); @@ -336,17 +311,8 @@ protected Type visitSymbolReference(SymbolReference node, StackableAstVisitorCon } @Override - protected Type visitQualifiedNameReference(QualifiedNameReference node, StackableAstVisitorContext context) + protected Type visitQualifiedNameReference(QualifiedNameReference node, StackableAstVisitorContext context) { - if (context.getContext().isInLambda()) { - LambdaArgumentDeclaration lambdaArgumentDeclaration = context.getContext().getNameToLambdaArgumentDeclarationMap().get(getOnlyElement(node.getName().getParts())); - if (lambdaArgumentDeclaration != null) { - lambdaArgumentReferences.put(node, lambdaArgumentDeclaration); - Type result = expressionTypes.get(lambdaArgumentDeclaration); - expressionTypes.put(node, result); - return result; - } - } return handleResolvedField(node, scope.resolveField(node, node.getName())); } @@ -358,20 +324,18 @@ private Type handleResolvedField(Expression node, ResolvedField resolvedField) } @Override - protected Type visitDereferenceExpression(DereferenceExpression node, StackableAstVisitorContext context) + protected Type visitDereferenceExpression(DereferenceExpression node, StackableAstVisitorContext context) { QualifiedName qualifiedName = DereferenceExpression.getQualifiedName(node); - if (!context.getContext().isInLambda()) { - // If this Dereference looks like column reference, try match it to column first. - if (qualifiedName != null) { - Optional resolvedField = scope.tryResolveField(node, qualifiedName); - if (resolvedField.isPresent()) { - return handleResolvedField(node, resolvedField.get()); - } - if (!scope.isColumnReference(qualifiedName)) { - throwMissingAttributeException(node, qualifiedName); - } + // If this Dereference looks like column reference, try match it to column first. + if (qualifiedName != null) { + Optional resolvedField = scope.tryResolveField(node, qualifiedName); + if (resolvedField.isPresent()) { + return handleResolvedField(node, resolvedField.get()); + } + if (!scope.isColumnReference(qualifiedName)) { + throwMissingAttributeException(node, qualifiedName); } } @@ -398,7 +362,7 @@ protected Type visitDereferenceExpression(DereferenceExpression node, StackableA } @Override - protected Type visitNotExpression(NotExpression node, StackableAstVisitorContext context) + protected Type visitNotExpression(NotExpression node, StackableAstVisitorContext context) { coerceType(context, node.getValue(), BOOLEAN, "Value of logical NOT expression"); @@ -407,7 +371,7 @@ protected Type visitNotExpression(NotExpression node, StackableAstVisitorContext } @Override - protected Type visitLogicalBinaryExpression(LogicalBinaryExpression node, StackableAstVisitorContext context) + protected Type visitLogicalBinaryExpression(LogicalBinaryExpression node, StackableAstVisitorContext context) { coerceType(context, node.getLeft(), BOOLEAN, "Left side of logical expression"); coerceType(context, node.getRight(), BOOLEAN, "Right side of logical expression"); @@ -417,14 +381,14 @@ protected Type visitLogicalBinaryExpression(LogicalBinaryExpression node, Stacka } @Override - protected Type visitComparisonExpression(ComparisonExpression node, StackableAstVisitorContext context) + protected Type visitComparisonExpression(ComparisonExpression node, StackableAstVisitorContext context) { OperatorType operatorType = OperatorType.valueOf(node.getType().name()); return getOperator(context, node, operatorType, node.getLeft(), node.getRight()); } @Override - protected Type visitIsNullPredicate(IsNullPredicate node, StackableAstVisitorContext context) + protected Type visitIsNullPredicate(IsNullPredicate node, StackableAstVisitorContext context) { process(node.getValue(), context); @@ -433,7 +397,7 @@ protected Type visitIsNullPredicate(IsNullPredicate node, StackableAstVisitorCon } @Override - protected Type visitIsNotNullPredicate(IsNotNullPredicate node, StackableAstVisitorContext context) + protected Type visitIsNotNullPredicate(IsNotNullPredicate node, StackableAstVisitorContext context) { process(node.getValue(), context); @@ -442,7 +406,7 @@ protected Type visitIsNotNullPredicate(IsNotNullPredicate node, StackableAstVisi } @Override - protected Type visitNullIfExpression(NullIfExpression node, StackableAstVisitorContext context) + protected Type visitNullIfExpression(NullIfExpression node, StackableAstVisitorContext context) { Type firstType = process(node.getFirst(), context); Type secondType = process(node.getSecond(), context); @@ -456,7 +420,7 @@ protected Type visitNullIfExpression(NullIfExpression node, StackableAstVisitorC } @Override - protected Type visitIfExpression(IfExpression node, StackableAstVisitorContext context) + protected Type visitIfExpression(IfExpression node, StackableAstVisitorContext context) { coerceType(context, node.getCondition(), BOOLEAN, "IF condition"); @@ -473,7 +437,7 @@ protected Type visitIfExpression(IfExpression node, StackableAstVisitorContext context) + protected Type visitSearchedCaseExpression(SearchedCaseExpression node, StackableAstVisitorContext context) { for (WhenClause whenClause : node.getWhenClauses()) { coerceType(context, whenClause.getOperand(), BOOLEAN, "CASE WHEN clause"); @@ -494,7 +458,7 @@ protected Type visitSearchedCaseExpression(SearchedCaseExpression node, Stackabl } @Override - protected Type visitSimpleCaseExpression(SimpleCaseExpression node, StackableAstVisitorContext context) + protected Type visitSimpleCaseExpression(SimpleCaseExpression node, StackableAstVisitorContext context) { for (WhenClause whenClause : node.getWhenClauses()) { coerceToSingleType(context, whenClause, "CASE operand type does not match WHEN clause operand type: %s vs %s", node.getOperand(), whenClause.getOperand()); @@ -525,7 +489,7 @@ private List getCaseResultExpressions(List whenClauses, } @Override - protected Type visitCoalesceExpression(CoalesceExpression node, StackableAstVisitorContext context) + protected Type visitCoalesceExpression(CoalesceExpression node, StackableAstVisitorContext context) { Type type = coerceToSingleType(context, "All COALESCE operands must be the same type: %s", node.getOperands()); @@ -534,7 +498,7 @@ protected Type visitCoalesceExpression(CoalesceExpression node, StackableAstVisi } @Override - protected Type visitArithmeticUnary(ArithmeticUnaryExpression node, StackableAstVisitorContext context) + protected Type visitArithmeticUnary(ArithmeticUnaryExpression node, StackableAstVisitorContext context) { switch (node.getSign()) { case PLUS: @@ -555,13 +519,13 @@ protected Type visitArithmeticUnary(ArithmeticUnaryExpression node, StackableAst } @Override - protected Type visitArithmeticBinary(ArithmeticBinaryExpression node, StackableAstVisitorContext context) + protected Type visitArithmeticBinary(ArithmeticBinaryExpression node, StackableAstVisitorContext context) { return getOperator(context, node, OperatorType.valueOf(node.getType().name()), node.getLeft(), node.getRight()); } @Override - protected Type visitLikePredicate(LikePredicate node, StackableAstVisitorContext context) + protected Type visitLikePredicate(LikePredicate node, StackableAstVisitorContext context) { Type valueType = getVarcharType(node.getValue(), context); Type patternType = getVarcharType(node.getPattern(), context); @@ -576,7 +540,7 @@ protected Type visitLikePredicate(LikePredicate node, StackableAstVisitorContext return BOOLEAN; } - private Type getVarcharType(Expression value, StackableAstVisitorContext context) + private Type getVarcharType(Expression value, StackableAstVisitorContext context) { Type type = process(value, context); if (!(type instanceof VarcharType)) { @@ -586,13 +550,13 @@ private Type getVarcharType(Expression value, StackableAstVisitorContext context) + protected Type visitSubscriptExpression(SubscriptExpression node, StackableAstVisitorContext context) { return getOperator(context, node, SUBSCRIPT, node.getBase(), node.getIndex()); } @Override - protected Type visitArrayConstructor(ArrayConstructor node, StackableAstVisitorContext context) + protected Type visitArrayConstructor(ArrayConstructor node, StackableAstVisitorContext context) { Type type = coerceToSingleType(context, "All ARRAY elements must be the same type: %s", node.getValues()); Type arrayType = typeManager.getParameterizedType(ARRAY.getName(), ImmutableList.of(TypeSignatureParameter.of(type.getTypeSignature()))); @@ -601,15 +565,14 @@ protected Type visitArrayConstructor(ArrayConstructor node, StackableAstVisitorC } @Override - protected Type visitStringLiteral(StringLiteral node, StackableAstVisitorContext context) + protected Type visitStringLiteral(StringLiteral node, StackableAstVisitorContext context) { VarcharType type = VarcharType.createVarcharType(SliceUtf8.countCodePoints(node.getSlice())); expressionTypes.put(node, type); return type; } - @Override - protected Type visitCharLiteral(CharLiteral node, StackableAstVisitorContext context) + protected Type visitCharLiteral(CharLiteral node, StackableAstVisitorContext context) { CharType type = CharType.createCharType(node.getValue().length()); expressionTypes.put(node, type); @@ -617,14 +580,14 @@ protected Type visitCharLiteral(CharLiteral node, StackableAstVisitorContext context) + protected Type visitBinaryLiteral(BinaryLiteral node, StackableAstVisitorContext context) { expressionTypes.put(node, VARBINARY); return VARBINARY; } @Override - protected Type visitLongLiteral(LongLiteral node, StackableAstVisitorContext context) + protected Type visitLongLiteral(LongLiteral node, StackableAstVisitorContext context) { if (node.getValue() >= Integer.MIN_VALUE && node.getValue() <= Integer.MAX_VALUE) { expressionTypes.put(node, INTEGER); @@ -636,14 +599,14 @@ protected Type visitLongLiteral(LongLiteral node, StackableAstVisitorContext context) + protected Type visitDoubleLiteral(DoubleLiteral node, StackableAstVisitorContext context) { expressionTypes.put(node, DOUBLE); return DOUBLE; } @Override - protected Type visitDecimalLiteral(DecimalLiteral node, StackableAstVisitorContext context) + protected Type visitDecimalLiteral(DecimalLiteral node, StackableAstVisitorContext context) { DecimalParseResult parseResult = Decimals.parse(node.getValue()); expressionTypes.put(node, parseResult.getType()); @@ -651,14 +614,14 @@ protected Type visitDecimalLiteral(DecimalLiteral node, StackableAstVisitorConte } @Override - protected Type visitBooleanLiteral(BooleanLiteral node, StackableAstVisitorContext context) + protected Type visitBooleanLiteral(BooleanLiteral node, StackableAstVisitorContext context) { expressionTypes.put(node, BOOLEAN); return BOOLEAN; } @Override - protected Type visitGenericLiteral(GenericLiteral node, StackableAstVisitorContext context) + protected Type visitGenericLiteral(GenericLiteral node, StackableAstVisitorContext context) { Type type = typeManager.getType(parseTypeSignature(node.getType())); if (type == null) { @@ -679,7 +642,7 @@ protected Type visitGenericLiteral(GenericLiteral node, StackableAstVisitorConte } @Override - protected Type visitTimeLiteral(TimeLiteral node, StackableAstVisitorContext context) + protected Type visitTimeLiteral(TimeLiteral node, StackableAstVisitorContext context) { Type type; if (timeHasTimeZone(node.getValue())) { @@ -693,7 +656,7 @@ protected Type visitTimeLiteral(TimeLiteral node, StackableAstVisitorContext context) + protected Type visitTimestampLiteral(TimestampLiteral node, StackableAstVisitorContext context) { try { parseTimestampLiteral(session.getTimeZoneKey(), node.getValue()); @@ -714,7 +677,7 @@ protected Type visitTimestampLiteral(TimestampLiteral node, StackableAstVisitorC } @Override - protected Type visitIntervalLiteral(IntervalLiteral node, StackableAstVisitorContext context) + protected Type visitIntervalLiteral(IntervalLiteral node, StackableAstVisitorContext context) { Type type; if (node.isYearToMonth()) { @@ -728,14 +691,14 @@ protected Type visitIntervalLiteral(IntervalLiteral node, StackableAstVisitorCon } @Override - protected Type visitNullLiteral(NullLiteral node, StackableAstVisitorContext context) + protected Type visitNullLiteral(NullLiteral node, StackableAstVisitorContext context) { expressionTypes.put(node, UNKNOWN); return UNKNOWN; } @Override - protected Type visitFunctionCall(FunctionCall node, StackableAstVisitorContext context) + protected Type visitFunctionCall(FunctionCall node, StackableAstVisitorContext context) { if (node.getWindow().isPresent()) { for (Expression expression : node.getWindow().get().getPartitionBy()) { @@ -779,44 +742,14 @@ protected Type visitFunctionCall(FunctionCall node, StackableAstVisitorContext argumentTypesBuilder = ImmutableList.builder(); + ImmutableList.Builder argumentTypes = ImmutableList.builder(); for (Expression expression : node.getArguments()) { - if (expression instanceof LambdaExpression) { - LambdaExpression lambdaExpression = (LambdaExpression) expression; - - // captures are not supported for now, use empty tuple descriptor - Expression lambdaBody = lambdaExpression.getBody(); - List lambdaArguments = lambdaExpression.getArguments(); - - argumentTypesBuilder.add(new TypeSignatureProvider( - types -> { - checkArgument(lambdaArguments.size() == types.size()); - ExpressionAnalyzer innerExpressionAnalyzer = new ExpressionAnalyzer( - functionRegistry, - typeManager, - statementAnalyzerFactory, - session, - symbolTypes, - parameters, - isDescribe); - Map nameToLambdaArgumentDeclarationMap = new HashMap<>(); - for (int i = 0; i < lambdaArguments.size(); i++) { - LambdaArgumentDeclaration lambdaArgument = lambdaArguments.get(i); - nameToLambdaArgumentDeclarationMap.put(lambdaArgument.getName(), lambdaArgument); - innerExpressionAnalyzer.getExpressionTypes().put(lambdaArgument, types.get(i)); - } - return new FunctionType(types, innerExpressionAnalyzer.analyze(lambdaBody, scope, Context.inLambda(nameToLambdaArgumentDeclarationMap))).getTypeSignature(); - })); - } - else { - argumentTypesBuilder.add(new TypeSignatureProvider(process(expression, context).getTypeSignature())); - } + argumentTypes.add(process(expression, context).getTypeSignature()); } - ImmutableList argumentTypes = argumentTypesBuilder.build(); Signature function; try { - function = functionRegistry.resolveFunction(node.getName(), argumentTypes); + function = functionRegistry.resolveFunction(node.getName(), argumentTypes.build()); } catch (PrestoException e) { if (e.getErrorCode().getCode() == StandardErrorCode.FUNCTION_NOT_FOUND.toErrorCode().getCode()) { @@ -830,32 +763,12 @@ protected Type visitFunctionCall(FunctionCall node, StackableAstVisitorContext lambdaArguments = lambdaExpression.getArguments(); - - Map nameToLambdaArgumentDeclarationMap = new HashMap<>(); - for (int j = 0; j < lambdaArguments.size(); j++) { - LambdaArgumentDeclaration lambdaArgument = lambdaArguments.get(j); - nameToLambdaArgumentDeclarationMap.put(lambdaArgument.getName(), lambdaArgument); - expressionTypes.put(lambdaArgument, functionType.getArgumentTypes().get(j)); - } - Type actualType = process(lambdaExpression.getBody(), new StackableAstVisitorContext<>(Context.inLambda(nameToLambdaArgumentDeclarationMap))); - - coerceType(lambdaExpression.getBody(), actualType, functionType.getReturnType(), format("Function %s argument %d", function, i)); - expressionTypes.put(lambdaExpression.getBody(), functionType.getReturnType()); - expressionTypes.put(lambdaExpression, functionType); - } - else { - Type actualType = typeManager.getType(argumentTypes.get(i).getTypeSignature()); - coerceType(expression, actualType, expectedType, format("Function %s argument %d", function, i)); + Type type = typeManager.getType(function.getArgumentTypes().get(i)); + requireNonNull(type, format("Type %s not found", function.getArgumentTypes().get(i))); + if (node.isDistinct() && !type.isComparable()) { + throw new SemanticException(TYPE_MISMATCH, node, "DISTINCT can only be applied to comparable types (actual: %s)", type); } + coerceType(context, expression, type, format("Function %s argument %d", function, i)); } resolvedFunctions.put(node, function); @@ -866,7 +779,7 @@ protected Type visitFunctionCall(FunctionCall node, StackableAstVisitorContext context) + protected Type visitAtTimeZone(AtTimeZone node, StackableAstVisitorContext context) { Type valueType = process(node.getValue(), context); process(node.getTimeZone(), context); @@ -886,7 +799,7 @@ else if (valueType.equals(TIMESTAMP)) { } @Override - protected Type visitParameter(Parameter node, StackableAstVisitorContext context) + protected Type visitParameter(Parameter node, StackableAstVisitorContext context) { if (isDescribe) { expressionTypes.put(node, UNKNOWN); @@ -905,7 +818,7 @@ protected Type visitParameter(Parameter node, StackableAstVisitorContext context) + protected Type visitExtract(Extract node, StackableAstVisitorContext context) { Type type = process(node.getExpression(), context); if (!isDateTimeType(type)) { @@ -932,24 +845,21 @@ private boolean isDateTimeType(Type type) } @Override - protected Type visitBetweenPredicate(BetweenPredicate node, StackableAstVisitorContext context) + protected Type visitBetweenPredicate(BetweenPredicate node, StackableAstVisitorContext context) { return getOperator(context, node, OperatorType.BETWEEN, node.getValue(), node.getMin(), node.getMax()); } @Override - public Type visitTryExpression(TryExpression node, StackableAstVisitorContext context) + public Type visitTryExpression(TryExpression node, StackableAstVisitorContext context) { - if (context.getContext().isInLambda()) { - throw new PrestoException(StandardErrorCode.NOT_SUPPORTED, "Try expression inside lambda expression is not support yet"); - } Type type = process(node.getInnerExpression(), context); expressionTypes.put(node, type); return type; } @Override - public Type visitCast(Cast node, StackableAstVisitorContext context) + public Type visitCast(Cast node, StackableAstVisitorContext context) { Type type = typeManager.getType(parseTypeSignature(node.getType())); if (type == null) { @@ -975,7 +885,7 @@ public Type visitCast(Cast node, StackableAstVisitorContext context) } @Override - protected Type visitInPredicate(InPredicate node, StackableAstVisitorContext context) + protected Type visitInPredicate(InPredicate node, StackableAstVisitorContext context) { Expression value = node.getValue(); process(value, context); @@ -999,7 +909,7 @@ else if (valueList instanceof SubqueryExpression) { } @Override - protected Type visitInListExpression(InListExpression node, StackableAstVisitorContext context) + protected Type visitInListExpression(InListExpression node, StackableAstVisitorContext context) { Type type = coerceToSingleType(context, "All IN list values must be the same type: %s", node.getValues()); @@ -1008,7 +918,7 @@ protected Type visitInListExpression(InListExpression node, StackableAstVisitorC } @Override - protected Type visitSubqueryExpression(SubqueryExpression node, StackableAstVisitorContext context) + protected Type visitSubqueryExpression(SubqueryExpression node, StackableAstVisitorContext context) { StatementAnalyzer analyzer = statementAnalyzerFactory.apply(node); Scope subqueryScope = createQueryBoundaryScope(); @@ -1040,7 +950,7 @@ else if (previousNode instanceof QuantifiedComparisonExpression) { } @Override - protected Type visitExists(ExistsPredicate node, StackableAstVisitorContext context) + protected Type visitExists(ExistsPredicate node, StackableAstVisitorContext context) { StatementAnalyzer analyzer = statementAnalyzerFactory.apply(node); Scope subqueryScope = createQueryBoundaryScope(); @@ -1054,7 +964,7 @@ protected Type visitExists(ExistsPredicate node, StackableAstVisitorContext context) + protected Type visitQuantifiedComparisonExpression(QuantifiedComparisonExpression node, StackableAstVisitorContext context) { Expression value = node.getValue(); process(value, context); @@ -1096,7 +1006,7 @@ private Scope createQueryBoundaryScope() } @Override - public Type visitFieldReference(FieldReference node, StackableAstVisitorContext context) + public Type visitFieldReference(FieldReference node, StackableAstVisitorContext context) { Type type = scope.getRelationType().getFieldByIndex(node.getFieldIndex()).getType(); expressionTypes.put(node, type); @@ -1104,26 +1014,18 @@ public Type visitFieldReference(FieldReference node, StackableAstVisitorContext< } @Override - protected Type visitLambdaExpression(LambdaExpression node, StackableAstVisitorContext context) - { - // visitFunctionCall looks through LambdaExpression if any function argument is a LambdaExpression, - // and handles the analysis of the LambdaExpression itself. - throw new SemanticException(STANDALONE_LAMBDA, node, "lambda expression should always be used inside a function"); - } - - @Override - protected Type visitExpression(Expression node, StackableAstVisitorContext context) + protected Type visitExpression(Expression node, StackableAstVisitorContext context) { throw new SemanticException(NOT_SUPPORTED, node, "not yet implemented: " + node.getClass().getName()); } @Override - protected Type visitNode(Node node, StackableAstVisitorContext context) + protected Type visitNode(Node node, StackableAstVisitorContext context) { throw new SemanticException(NOT_SUPPORTED, node, "not yet implemented: " + node.getClass().getName()); } - private Type getOperator(StackableAstVisitorContext context, Expression node, OperatorType operatorType, Expression... arguments) + private Type getOperator(StackableAstVisitorContext context, Expression node, OperatorType operatorType, Expression... arguments) { ImmutableList.Builder argumentTypes = ImmutableList.builder(); for (Expression expression : arguments) { @@ -1156,8 +1058,9 @@ private Type getOperator(StackableAstVisitorContext context, Expression return type; } - private void coerceType(Expression expression, Type actualType, Type expectedType, String message) + private void coerceType(StackableAstVisitorContext context, Expression expression, Type expectedType, String message) { + Type actualType = process(expression, context); if (!actualType.equals(expectedType)) { if (!typeManager.canCoerce(actualType, expectedType)) { throw new SemanticException(TYPE_MISMATCH, expression, message + " must evaluate to a %s (actual: %s)", expectedType, actualType); @@ -1169,13 +1072,7 @@ private void coerceType(Expression expression, Type actualType, Type expectedTyp } } - private void coerceType(StackableAstVisitorContext context, Expression expression, Type expectedType, String message) - { - Type actualType = process(expression, context); - coerceType(expression, actualType, expectedType, message); - } - - private Type coerceToSingleType(StackableAstVisitorContext context, Node node, String message, Expression first, Expression second) + private Type coerceToSingleType(StackableAstVisitorContext context, Node node, String message, Expression first, Expression second) { Type firstType = null; if (first != null) { @@ -1214,7 +1111,7 @@ private Type coerceToSingleType(StackableAstVisitorContext context, Nod throw new SemanticException(TYPE_MISMATCH, node, message, firstType, secondType); } - private Type coerceToSingleType(StackableAstVisitorContext context, String message, List expressions) + private Type coerceToSingleType(StackableAstVisitorContext context, String message, List expressions) { // determine super type Type superType = UNKNOWN; @@ -1244,37 +1141,6 @@ private Type coerceToSingleType(StackableAstVisitorContext context, Str } } - private static class Context - { - private final Map nameToLambdaArgumentDeclarationMap; - - private Context(Map nameToLambdaArgumentDeclarationMap) - { - this.nameToLambdaArgumentDeclarationMap = nameToLambdaArgumentDeclarationMap; - } - - public static Context notInLambda() - { - return new Context(null); - } - - public static Context inLambda(Map nameToLambdaArgumentDeclarationMap) - { - return new Context(requireNonNull(nameToLambdaArgumentDeclarationMap, "nameToLambdaArgumentDeclarationMap is null")); - } - - public boolean isInLambda() - { - return nameToLambdaArgumentDeclarationMap != null; - } - - public Map getNameToLambdaArgumentDeclarationMap() - { - checkState(isInLambda()); - return nameToLambdaArgumentDeclarationMap; - } - } - public static IdentityHashMap getExpressionTypes( Session session, Metadata metadata, @@ -1399,8 +1265,7 @@ private static ExpressionAnalysis analyzeExpressions( analyzer.getExistsSubqueries(), analyzer.getColumnReferences(), analyzer.getTypeOnlyCoercions(), - analyzer.getQuantifiedComparisons(), - analyzer.getLambdaArgumentReferences()); + analyzer.getQuantifiedComparisons()); } public static ExpressionAnalysis analyzeExpression( @@ -1424,7 +1289,6 @@ public static ExpressionAnalysis analyzeExpression( analysis.addCoercions(expressionCoercions, typeOnlyCoercions); analysis.addFunctionSignatures(resolvedFunctions); analysis.addColumnReferences(analyzer.getColumnReferences()); - analysis.addLambdaArgumentReferences(analyzer.getLambdaArgumentReferences()); return new ExpressionAnalysis( expressionTypes, @@ -1434,8 +1298,7 @@ public static ExpressionAnalysis analyzeExpression( analyzer.getExistsSubqueries(), analyzer.getColumnReferences(), analyzer.getTypeOnlyCoercions(), - analyzer.getQuantifiedComparisons(), - analyzer.getLambdaArgumentReferences()); + analyzer.getQuantifiedComparisons()); } public static ExpressionAnalyzer create( diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java index e2ea9f58efef..b3fd04c5e1fb 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/FeaturesConfig.java @@ -52,7 +52,6 @@ public static class ProcessingOptimization private boolean optimizeSingleDistinct = true; private boolean pushTableWriteThroughUnion = true; private boolean legacyArrayAgg; - private boolean legacyOrderBy; private boolean optimizeMixedDistinctAggregations; private String processingOptimization = ProcessingOptimization.DISABLED; @@ -108,18 +107,6 @@ public boolean isLegacyArrayAgg() return legacyArrayAgg; } - @Config("deprecated.legacy-order-by") - public FeaturesConfig setLegacyOrderBy(boolean value) - { - this.legacyOrderBy = value; - return this; - } - - public boolean isLegacyOrderBy() - { - return legacyOrderBy; - } - @Config("distributed-joins-enabled") public FeaturesConfig setDistributedJoinsEnabled(boolean distributedJoinsEnabled) { diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/QueryExplainer.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/QueryExplainer.java index 41366aa15a93..392c7d0c3ea8 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/QueryExplainer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/QueryExplainer.java @@ -143,6 +143,6 @@ private SubPlan getDistributedPlan(Session session, Statement statement, List resolveFields(QualifiedName name) .collect(toImmutableList()); } - public boolean canResolve(QualifiedName name) + public Predicate canResolvePredicate() { - return !resolveFields(name).isEmpty(); + return input -> !resolveFields(input).isEmpty(); } /** diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/Scope.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/Scope.java index b3f06c1b372e..60fcb0c26afa 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/Scope.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/Scope.java @@ -82,7 +82,7 @@ public Optional tryResolveField(Expression expression) return Optional.empty(); } - private static QualifiedName asQualifiedName(Expression expression) + private QualifiedName asQualifiedName(Expression expression) { QualifiedName name = null; if (expression instanceof QualifiedNameReference) { @@ -150,7 +150,7 @@ public boolean isColumnReference(QualifiedName name) return false; } - private static boolean isColumnReference(QualifiedName name, RelationType relation) + private boolean isColumnReference(QualifiedName name, RelationType relation) { while (name.getPrefix().isPresent()) { name = name.getPrefix().get(); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/SemanticErrorCode.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/SemanticErrorCode.java index ca869f7e9cc2..2914c0d15a06 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/SemanticErrorCode.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/SemanticErrorCode.java @@ -83,6 +83,4 @@ public enum SemanticErrorCode AMBIGUOUS_FUNCTION_CALL, INVALID_PARAMETER_USAGE, - - STANDALONE_LAMBDA, } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java index 2966fa465afb..221239002dd6 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/StatementAnalyzer.java @@ -14,7 +14,6 @@ package com.facebook.presto.sql.analyzer; import com.facebook.presto.Session; -import com.facebook.presto.SystemSessionProperties; import com.facebook.presto.metadata.FunctionKind; import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.QualifiedObjectName; @@ -41,31 +40,21 @@ import com.facebook.presto.sql.tree.AddColumn; import com.facebook.presto.sql.tree.AliasedRelation; import com.facebook.presto.sql.tree.AllColumns; -import com.facebook.presto.sql.tree.Call; -import com.facebook.presto.sql.tree.Commit; import com.facebook.presto.sql.tree.ComparisonExpression; -import com.facebook.presto.sql.tree.CreateSchema; import com.facebook.presto.sql.tree.CreateTable; import com.facebook.presto.sql.tree.CreateTableAsSelect; import com.facebook.presto.sql.tree.CreateView; -import com.facebook.presto.sql.tree.Deallocate; +import com.facebook.presto.sql.tree.DataDefinitionStatement; import com.facebook.presto.sql.tree.DefaultTraversalVisitor; import com.facebook.presto.sql.tree.Delete; import com.facebook.presto.sql.tree.DereferenceExpression; -import com.facebook.presto.sql.tree.DropSchema; -import com.facebook.presto.sql.tree.DropTable; -import com.facebook.presto.sql.tree.DropView; import com.facebook.presto.sql.tree.Except; -import com.facebook.presto.sql.tree.Execute; import com.facebook.presto.sql.tree.Explain; import com.facebook.presto.sql.tree.ExplainType; import com.facebook.presto.sql.tree.Expression; -import com.facebook.presto.sql.tree.ExpressionRewriter; -import com.facebook.presto.sql.tree.ExpressionTreeRewriter; import com.facebook.presto.sql.tree.FieldReference; import com.facebook.presto.sql.tree.FrameBound; import com.facebook.presto.sql.tree.FunctionCall; -import com.facebook.presto.sql.tree.Grant; import com.facebook.presto.sql.tree.GroupingElement; import com.facebook.presto.sql.tree.Insert; import com.facebook.presto.sql.tree.Intersect; @@ -76,18 +65,11 @@ import com.facebook.presto.sql.tree.LongLiteral; import com.facebook.presto.sql.tree.NaturalJoin; import com.facebook.presto.sql.tree.Node; -import com.facebook.presto.sql.tree.Prepare; import com.facebook.presto.sql.tree.QualifiedName; import com.facebook.presto.sql.tree.QualifiedNameReference; import com.facebook.presto.sql.tree.Query; import com.facebook.presto.sql.tree.QuerySpecification; import com.facebook.presto.sql.tree.Relation; -import com.facebook.presto.sql.tree.RenameColumn; -import com.facebook.presto.sql.tree.RenameSchema; -import com.facebook.presto.sql.tree.RenameTable; -import com.facebook.presto.sql.tree.ResetSession; -import com.facebook.presto.sql.tree.Revoke; -import com.facebook.presto.sql.tree.Rollback; import com.facebook.presto.sql.tree.Row; import com.facebook.presto.sql.tree.SampledRelation; import com.facebook.presto.sql.tree.SelectItem; @@ -128,9 +110,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.stream.Collectors; -import static com.facebook.presto.SystemSessionProperties.LEGACY_ORDER_BY; import static com.facebook.presto.metadata.FunctionKind.AGGREGATE; import static com.facebook.presto.metadata.FunctionKind.WINDOW; import static com.facebook.presto.metadata.MetadataUtil.createQualifiedObjectName; @@ -164,7 +144,6 @@ import static com.facebook.presto.sql.analyzer.SemanticErrorCode.VIEW_IS_STALE; import static com.facebook.presto.sql.analyzer.SemanticErrorCode.VIEW_PARSE_ERROR; import static com.facebook.presto.sql.analyzer.SemanticErrorCode.WILDCARD_WITHOUT_FROM; -import static com.facebook.presto.sql.analyzer.TypeSignatureProvider.fromTypeSignatures; import static com.facebook.presto.sql.planner.ExpressionInterpreter.expressionOptimizer; import static com.facebook.presto.sql.tree.ComparisonExpressionType.EQUAL; import static com.facebook.presto.sql.tree.ExplainType.Type.DISTRIBUTED; @@ -384,127 +363,37 @@ protected Scope visitCreateView(CreateView node, Scope scope) validateColumns(node, queryScope.getRelationType()); - return createScope(node, scope, emptyList()); + return createScope(node, scope, queryScope.getRelationType()); } @Override - protected Scope visitSetSession(SetSession node, Scope scope) + protected Scope visitDataDefinitionStatement(DataDefinitionStatement node, Scope scope) { return createScope(node, scope, emptyList()); } @Override - protected Scope visitResetSession(ResetSession node, Scope scope) + protected Scope visitSetSession(SetSession node, Scope scope) { - return createScope(node, scope, emptyList()); + return visitDataDefinitionStatement(node, scope); } @Override protected Scope visitAddColumn(AddColumn node, Scope scope) { - return createScope(node, scope, emptyList()); - } - - @Override - protected Scope visitCreateSchema(CreateSchema node, Scope scope) - { - return createScope(node, scope, emptyList()); - } - - @Override - protected Scope visitDropSchema(DropSchema node, Scope scope) - { - return createScope(node, scope, emptyList()); - } - - @Override - protected Scope visitRenameSchema(RenameSchema node, Scope scope) - { - return createScope(node, scope, emptyList()); + return visitDataDefinitionStatement(node, scope); } @Override protected Scope visitCreateTable(CreateTable node, Scope scope) { - return createScope(node, scope, emptyList()); - } - - @Override - protected Scope visitDropTable(DropTable node, Scope scope) - { - return createScope(node, scope, emptyList()); - } - - @Override - protected Scope visitRenameTable(RenameTable node, Scope scope) - { - return createScope(node, scope, emptyList()); - } - - @Override - protected Scope visitRenameColumn(RenameColumn node, Scope scope) - { - return createScope(node, scope, emptyList()); - } - - @Override - protected Scope visitDropView(DropView node, Scope scope) - { - return createScope(node, scope, emptyList()); + return visitDataDefinitionStatement(node, scope); } @Override protected Scope visitStartTransaction(StartTransaction node, Scope scope) { - return createScope(node, scope, emptyList()); - } - - @Override - protected Scope visitCommit(Commit node, Scope scope) - { - return createScope(node, scope, emptyList()); - } - - @Override - protected Scope visitRollback(Rollback node, Scope scope) - { - return createScope(node, scope, emptyList()); - } - - @Override - protected Scope visitPrepare(Prepare node, Scope scope) - { - return createScope(node, scope, emptyList()); - } - - @Override - protected Scope visitDeallocate(Deallocate node, Scope scope) - { - return createScope(node, scope, emptyList()); - } - - @Override - protected Scope visitExecute(Execute node, Scope scope) - { - return createScope(node, scope, emptyList()); - } - - @Override - protected Scope visitGrant(Grant node, Scope scope) - { - return createScope(node, scope, emptyList()); - } - - @Override - protected Scope visitRevoke(Revoke node, Scope scope) - { - return createScope(node, scope, emptyList()); - } - - @Override - protected Scope visitCall(Call node, Scope scope) - { - return createScope(node, scope, emptyList()); + return visitDataDefinitionStatement(node, scope); } private static void validateColumns(Statement node, RelationType descriptor) @@ -790,12 +679,7 @@ protected Scope visitQuerySpecification(QuerySpecification node, Scope scope) List orderByExpressions = analyzeOrderBy(node, sourceScope, outputScope, outputExpressions); analyzeHaving(node, sourceScope); - List expressions = new ArrayList<>(); - expressions.addAll(outputExpressions); - expressions.addAll(orderByExpressions); - node.getHaving().ifPresent(expressions::add); - - analyzeAggregations(node, sourceScope, groupByExpressions, analysis.getColumnReferences(), expressions); + analyzeAggregations(node, sourceScope, groupByExpressions, outputExpressions, orderByExpressions, analysis.getColumnReferences()); analyzeWindowFunctions(node, outputExpressions, orderByExpressions); return outputScope; @@ -987,11 +871,11 @@ else if (criteria instanceof JoinOn) { Expression leftExpression = null; Expression rightExpression = null; - if (firstDependencies.stream().allMatch(left.getRelationType()::canResolve) && secondDependencies.stream().allMatch(right.getRelationType()::canResolve)) { + if (firstDependencies.stream().allMatch(left.getRelationType().canResolvePredicate()) && secondDependencies.stream().allMatch(right.getRelationType().canResolvePredicate())) { leftExpression = conjunctFirst; rightExpression = conjunctSecond; } - else if (firstDependencies.stream().allMatch(right.getRelationType()::canResolve) && secondDependencies.stream().allMatch(left.getRelationType()::canResolve)) { + else if (firstDependencies.stream().allMatch(right.getRelationType().canResolvePredicate()) && secondDependencies.stream().allMatch(left.getRelationType().canResolvePredicate())) { leftExpression = conjunctSecond; rightExpression = conjunctFirst; } @@ -1176,7 +1060,7 @@ private void analyzeWindowFunctions(QuerySpecification node, List ou List argumentTypes = Lists.transform(windowFunction.getArguments(), expression -> analysis.getType(expression).getTypeSignature()); - FunctionKind kind = metadata.getFunctionRegistry().resolveFunction(windowFunction.getName(), fromTypeSignatures(argumentTypes)).getKind(); + FunctionKind kind = metadata.getFunctionRegistry().resolveFunction(windowFunction.getName(), argumentTypes).getKind(); if (kind != AGGREGATE && kind != WINDOW) { throw new SemanticException(MUST_BE_WINDOW_FUNCTION, node, "Not a window function: %s", windowFunction.getName()); } @@ -1231,76 +1115,6 @@ private void analyzeHaving(QuerySpecification node, Scope scope) } private List analyzeOrderBy(QuerySpecification node, Scope sourceScope, Scope outputScope, List outputExpressions) - { - if (SystemSessionProperties.isLegacyOrderByEnabled(session)) { - return legacyAnalyzeOrderBy(node, sourceScope, outputScope, outputExpressions); - } - - List items = node.getOrderBy(); - - ImmutableList.Builder orderByExpressionsBuilder = ImmutableList.builder(); - - if (!items.isEmpty()) { - for (SortItem item : items) { - Expression expression = item.getSortKey(); - - Expression orderByExpression; - if (expression instanceof LongLiteral) { - // this is an ordinal in the output tuple - - long ordinal = ((LongLiteral) expression).getValue(); - if (ordinal < 1 || ordinal > outputExpressions.size()) { - throw new SemanticException(INVALID_ORDINAL, expression, "ORDER BY position %s is not in select list", ordinal); - } - - int field = Ints.checkedCast(ordinal - 1); - Type type = outputScope.getRelationType().getFieldByIndex(field).getType(); - if (!type.isOrderable()) { - throw new SemanticException(TYPE_MISMATCH, node, "The type of expression in position %s is not orderable (actual: %s), and therefore cannot be used in ORDER BY", ordinal, type); - } - - orderByExpression = outputExpressions.get(field); - } - else { - // Analyze the original expression using a synthetic scope (which delegates to the source scope for any missing name) - // to catch any semantic errors (due to type mismatch, etc) - Scope synthetic = Scope.builder() - .withParent(sourceScope) - .markQueryBoundary() // this is needed because of how the field resolution walks scopes - .withRelationType(outputScope.getRelationType()) - .build(); - - analyzeExpression(expression, synthetic); - - orderByExpression = ExpressionTreeRewriter.rewriteWith(new OrderByExpressionRewriter(extractNamedOutputExpressions(node)), expression); - - ExpressionAnalysis expressionAnalysis = analyzeExpression(orderByExpression, sourceScope); - analysis.recordSubqueries(node, expressionAnalysis); - } - - Type type = analysis.getType(orderByExpression); - if (!type.isOrderable()) { - throw new SemanticException(TYPE_MISMATCH, node, "Type %s is not orderable, and therefore cannot be used in ORDER BY: %s", type, expression); - } - - orderByExpressionsBuilder.add(orderByExpression); - } - } - - List orderByExpressions = orderByExpressionsBuilder.build(); - analysis.setOrderByExpressions(node, orderByExpressions); - - if (node.getSelect().isDistinct() && !outputExpressions.containsAll(orderByExpressions)) { - throw new SemanticException(ORDER_BY_MUST_BE_IN_SELECT, node.getSelect(), "For SELECT DISTINCT, ORDER BY expressions must appear in select list"); - } - return orderByExpressions; - } - - /** - * Preserve the old column resolution behavior for ORDER BY while we transition workloads to new semantics - * TODO: remove this - */ - private List legacyAnalyzeOrderBy(QuerySpecification node, Scope sourceScope, Scope outputScope, List outputExpressions) { List items = node.getOrderBy(); @@ -1380,62 +1194,6 @@ else if (expression instanceof LongLiteral) { return orderByExpressions; } - private static Multimap extractNamedOutputExpressions(QuerySpecification node) - { - // Compute aliased output terms so we can resolve order by expressions against them first - ImmutableMultimap.Builder assignments = ImmutableMultimap.builder(); - for (SelectItem item : node.getSelect().getSelectItems()) { - if (item instanceof SingleColumn) { - SingleColumn column = (SingleColumn) item; - Optional alias = column.getAlias(); - if (alias.isPresent()) { - assignments.put(QualifiedName.of(alias.get()), column.getExpression()); // TODO: need to know if alias was quoted - } - else if (column.getExpression() instanceof QualifiedNameReference) { - assignments.put(((QualifiedNameReference) column.getExpression()).getName(), column.getExpression()); - } - } - } - - return assignments.build(); - } - - private static class OrderByExpressionRewriter - extends ExpressionRewriter - { - private final Multimap assignments; - - public OrderByExpressionRewriter(Multimap assignments) - { - this.assignments = assignments; - } - - @Override - public Expression rewriteQualifiedNameReference(QualifiedNameReference reference, Void context, ExpressionTreeRewriter treeRewriter) - { - if (reference.getName().getPrefix().isPresent()) { - return reference; - } - - // if this is a simple name reference, try to resolve against output columns - QualifiedName name = reference.getName(); - Set expressions = assignments.get(name) - .stream() - .collect(Collectors.toSet()); - - if (expressions.size() > 1) { - throw new SemanticException(AMBIGUOUS_ATTRIBUTE, reference, "'%s' in ORDER BY is ambiguous", name.getSuffix()); - } - - if (expressions.size() == 1) { - return Iterables.getOnlyElement(expressions); - } - - // otherwise, couldn't resolve name against output aliases, so fall through... - return reference; - } - } - private List> analyzeGroupBy(QuerySpecification node, Scope scope, List outputExpressions) { List> computedGroupingSets = ImmutableList.of(); // empty list = no aggregations @@ -1449,7 +1207,7 @@ private List> analyzeGroupBy(QuerySpecification node, Scope sco computedGroupingSets = computeGroupingSetsCrossProduct(enumeratedGroupingSets, node.getGroupBy().get().isDistinct()); checkState(!computedGroupingSets.isEmpty(), "computed grouping sets cannot be empty"); } - else if (hasAggregates(node)) { + else if (!extractAggregates(node).isEmpty()) { // if there are aggregates, but no group by, create a grand total grouping set (global aggregation) computedGroupingSets = ImmutableList.of(ImmutableSet.of()); } @@ -1462,7 +1220,7 @@ else if (hasAggregates(node)) { return analyzedGroupingSets; } - private static List> computeGroupingSetsCrossProduct(List>> enumeratedGroupingSets, boolean isDistinct) + private List> computeGroupingSetsCrossProduct(List>> enumeratedGroupingSets, boolean isDistinct) { checkState(!enumeratedGroupingSets.isEmpty(), "enumeratedGroupingSets cannot be empty"); @@ -1662,14 +1420,11 @@ private void analyzeAggregations( QuerySpecification node, Scope scope, List> groupingSets, - Set columnReferences, - List expressions) + List outputExpressions, + List orderByExpressions, + Set columnReferences) { - AggregateExtractor extractor = new AggregateExtractor(metadata); - for (Expression expression : expressions) { - extractor.process(expression); - } - analysis.setAggregates(node, extractor.getAggregates()); + extractAggregates(node); // is this an aggregation query? if (!groupingSets.isEmpty()) { @@ -1683,28 +1438,37 @@ private void analyzeAggregations( .distinct() .collect(toImmutableList()); - for (Expression expression : expressions) { + for (Expression expression : Iterables.concat(outputExpressions, orderByExpressions)) { verifyAggregations(distinctGroupingColumns, scope, expression, columnReferences); } + + if (node.getHaving().isPresent()) { + verifyAggregations(distinctGroupingColumns, scope, node.getHaving().get(), columnReferences); + } } } - private boolean hasAggregates(QuerySpecification node) + private List extractAggregates(QuerySpecification node) { AggregateExtractor extractor = new AggregateExtractor(metadata); + for (SelectItem item : node.getSelect().getSelectItems()) { + if (item instanceof SingleColumn) { + extractor.process(((SingleColumn) item).getExpression(), null); + } + } - node.getSelect() - .getSelectItems().stream() - .filter(SingleColumn.class::isInstance) - .forEach(extractor::process); + for (SortItem item : node.getOrderBy()) { + extractor.process(item.getSortKey(), null); + } - node.getOrderBy().stream() - .forEach(extractor::process); + if (node.getHaving().isPresent()) { + extractor.process(node.getHaving().get(), null); + } - node.getHaving() - .ifPresent(extractor::process); + List aggregates = extractor.getAggregates(); + analysis.setAggregates(node, aggregates); - return !extractor.getAggregates().isEmpty(); + return aggregates; } private void verifyAggregations( @@ -1713,7 +1477,7 @@ private void verifyAggregations( Expression expression, Set columnReferences) { - AggregationAnalyzer analyzer = new AggregationAnalyzer(groupByExpressions, metadata, scope, columnReferences, analysis.getParameters(), analysis.isDescribe()); + AggregationAnalyzer analyzer = new AggregationAnalyzer(groupByExpressions, metadata, scope, columnReferences, analysis.getParameters()); analyzer.analyze(expression); } @@ -1744,7 +1508,6 @@ private RelationType analyzeView(Query query, QualifiedObjectName name, Optional .setRemoteUserAddress(session.getRemoteUserAddress().orElse(null)) .setUserAgent(session.getUserAgent().orElse(null)) .setStartTime(session.getStartTime()) - .setSystemProperty(LEGACY_ORDER_BY, session.getSystemProperty(LEGACY_ORDER_BY, Boolean.class).toString()) .build(); StatementAnalyzer analyzer = new StatementAnalyzer(analysis, metadata, sqlParser, viewAccessControl, viewSession); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/TypeSignatureProvider.java b/presto-main/src/main/java/com/facebook/presto/sql/analyzer/TypeSignatureProvider.java deleted file mode 100644 index 857f7bd549db..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/sql/analyzer/TypeSignatureProvider.java +++ /dev/null @@ -1,87 +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.sql.analyzer; - -import com.facebook.presto.spi.type.Type; -import com.facebook.presto.spi.type.TypeSignature; -import com.google.common.collect.ImmutableList; - -import java.util.List; -import java.util.function.Function; - -import static com.facebook.presto.util.ImmutableCollectors.toImmutableList; -import static com.google.common.base.Preconditions.checkState; -import static java.util.Objects.requireNonNull; - -public class TypeSignatureProvider -{ - // hasDependency field exists primarily to make manipulating types without dependencies easy, - // and to make toString more friendly. - private final boolean hasDependency; - private final Function, TypeSignature> typeSignatureResolver; - - public TypeSignatureProvider(TypeSignature typeSignature) - { - this.hasDependency = false; - this.typeSignatureResolver = ignored -> typeSignature; - } - - public TypeSignatureProvider(Function, TypeSignature> typeSignatureResolver) - { - this.hasDependency = true; - this.typeSignatureResolver = requireNonNull(typeSignatureResolver, "typeSignatureResolver is null"); - } - - public boolean hasDependency() - { - return hasDependency; - } - - public TypeSignature getTypeSignature() - { - checkState(!hasDependency); - return typeSignatureResolver.apply(ImmutableList.of()); - } - - public TypeSignature getTypeSignature(List boundTypeParameters) - { - checkState(hasDependency); - return typeSignatureResolver.apply(boundTypeParameters); - } - - public static List fromTypes(List types) - { - return types.stream() - .map(Type::getTypeSignature) - .map(TypeSignatureProvider::new) - .collect(toImmutableList()); - } - - public static List fromTypeSignatures(List typeSignatures) - { - return typeSignatures.stream() - .map(TypeSignatureProvider::new) - .collect(toImmutableList()); - } - - @Override - public String toString() - { - if (hasDependency) { - return super.toString(); - } - return getTypeSignature().toString(); - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/BytecodeExpressionVisitor.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/BytecodeExpressionVisitor.java index aabbc2bc4851..a7cd86f82527 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/BytecodeExpressionVisitor.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/gen/BytecodeExpressionVisitor.java @@ -15,19 +15,17 @@ import com.facebook.presto.bytecode.BytecodeBlock; import com.facebook.presto.bytecode.BytecodeNode; +import com.facebook.presto.bytecode.MethodDefinition; import com.facebook.presto.bytecode.Scope; import com.facebook.presto.metadata.FunctionRegistry; import com.facebook.presto.sql.relational.CallExpression; import com.facebook.presto.sql.relational.ConstantExpression; import com.facebook.presto.sql.relational.InputReferenceExpression; -import com.facebook.presto.sql.relational.LambdaDefinitionExpression; import com.facebook.presto.sql.relational.RowExpressionVisitor; -import com.facebook.presto.sql.relational.VariableReferenceExpression; -import java.lang.invoke.MethodHandle; +import java.util.Map; import static com.facebook.presto.bytecode.expression.BytecodeExpressions.constantTrue; -import static com.facebook.presto.bytecode.expression.BytecodeExpressions.getStatic; import static com.facebook.presto.bytecode.instruction.Constant.loadBoolean; import static com.facebook.presto.bytecode.instruction.Constant.loadDouble; import static com.facebook.presto.bytecode.instruction.Constant.loadFloat; @@ -45,7 +43,6 @@ import static com.facebook.presto.sql.relational.Signatures.ROW_CONSTRUCTOR; import static com.facebook.presto.sql.relational.Signatures.SWITCH; import static com.facebook.presto.sql.relational.Signatures.TRY; -import static com.google.common.base.Preconditions.checkState; public class BytecodeExpressionVisitor implements RowExpressionVisitor @@ -54,20 +51,20 @@ public class BytecodeExpressionVisitor private final CachedInstanceBinder cachedInstanceBinder; private final RowExpressionVisitor fieldReferenceCompiler; private final FunctionRegistry registry; - private final PreGeneratedExpressions preGeneratedExpressions; + private final Map tryExpressionsMap; public BytecodeExpressionVisitor( CallSiteBinder callSiteBinder, CachedInstanceBinder cachedInstanceBinder, RowExpressionVisitor fieldReferenceCompiler, FunctionRegistry registry, - PreGeneratedExpressions preGeneratedExpressions) + Map tryExpressionsMap) { this.callSiteBinder = callSiteBinder; this.cachedInstanceBinder = cachedInstanceBinder; this.fieldReferenceCompiler = fieldReferenceCompiler; this.registry = registry; - this.preGeneratedExpressions = preGeneratedExpressions; + this.tryExpressionsMap = tryExpressionsMap; } @Override @@ -92,7 +89,7 @@ public BytecodeNode visitCall(CallExpression call, final Scope scope) generator = new SwitchCodeGenerator(); break; case TRY: - generator = new TryCodeGenerator(preGeneratedExpressions.getTryMethodMap()); + generator = new TryCodeGenerator(tryExpressionsMap); break; // functions that take null as input case IS_NULL: @@ -184,20 +181,4 @@ public BytecodeNode visitInputReference(InputReferenceExpression node, Scope sco { return fieldReferenceCompiler.visitInputReference(node, scope); } - - @Override - public BytecodeNode visitLambda(LambdaDefinitionExpression lambda, Scope scope) - { - checkState(preGeneratedExpressions.getLambdaFieldMap().containsKey(lambda), "lambda expressions map does not contain this lambda definition"); - - return getStatic(preGeneratedExpressions.getLambdaFieldMap().get(lambda)) - .invoke("bindTo", MethodHandle.class, scope.getThis().cast(Object.class)) - .invoke("bindTo", MethodHandle.class, scope.getVariable("session").cast(Object.class)); - } - - @Override - public BytecodeNode visitVariableReference(VariableReferenceExpression reference, Scope scope) - { - return fieldReferenceCompiler.visitVariableReference(reference, scope); - } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/CursorProcessorCompiler.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/CursorProcessorCompiler.java index cdce084cef2d..656a41901aad 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/CursorProcessorCompiler.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/gen/CursorProcessorCompiler.java @@ -16,7 +16,6 @@ import com.facebook.presto.bytecode.BytecodeBlock; import com.facebook.presto.bytecode.BytecodeNode; import com.facebook.presto.bytecode.ClassDefinition; -import com.facebook.presto.bytecode.FieldDefinition; import com.facebook.presto.bytecode.MethodDefinition; import com.facebook.presto.bytecode.Parameter; import com.facebook.presto.bytecode.Scope; @@ -34,18 +33,15 @@ import com.facebook.presto.sql.relational.CallExpression; import com.facebook.presto.sql.relational.ConstantExpression; import com.facebook.presto.sql.relational.InputReferenceExpression; -import com.facebook.presto.sql.relational.LambdaDefinitionExpression; import com.facebook.presto.sql.relational.RowExpression; import com.facebook.presto.sql.relational.RowExpressionVisitor; -import com.facebook.presto.sql.relational.Signatures; -import com.facebook.presto.sql.relational.VariableReferenceExpression; -import com.google.common.base.VerifyException; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.primitives.Primitives; import io.airlift.slice.Slice; import java.util.List; +import java.util.Map; import static com.facebook.presto.bytecode.Access.PUBLIC; import static com.facebook.presto.bytecode.Access.a; @@ -53,9 +49,7 @@ import static com.facebook.presto.bytecode.Parameter.arg; import static com.facebook.presto.bytecode.ParameterizedType.type; import static com.facebook.presto.sql.gen.BytecodeUtils.generateWrite; -import static com.facebook.presto.sql.gen.LambdaAndTryExpressionExtractor.extractLambdaAndTryExpressions; import static com.facebook.presto.sql.gen.TryCodeGenerator.defineTryMethod; -import static com.google.common.base.Verify.verify; import static java.lang.String.format; public class CursorProcessorCompiler @@ -89,7 +83,7 @@ public void generateMethods(ClassDefinition classDefinition, CallSiteBinder call constructorBody.ret(); } - private static void generateProcessMethod(ClassDefinition classDefinition, int projections) + private void generateProcessMethod(ClassDefinition classDefinition, int projections) { Parameter session = arg("session", ConnectorSession.class); Parameter cursor = arg("cursor", RecordCursor.class); @@ -180,76 +174,57 @@ private static void generateProcessMethod(ClassDefinition classDefinition, int p .retInt(); } - private PreGeneratedExpressions generateMethodsForLambdaAndTry( + private Map generateTryMethods( ClassDefinition containerClassDefinition, CallSiteBinder callSiteBinder, CachedInstanceBinder cachedInstanceBinder, RowExpression projection, String methodPrefix) { - List lambdaAndTryExpressions = extractLambdaAndTryExpressions(projection); + TryExpressionExtractor tryExtractor = new TryExpressionExtractor(); + projection.accept(tryExtractor, null); + List tryExpressions = tryExtractor.getTryExpressionsPreOrder(); ImmutableMap.Builder tryMethodMap = ImmutableMap.builder(); - ImmutableMap.Builder lambdaFieldMap = ImmutableMap.builder(); - - for (int i = 0; i < lambdaAndTryExpressions.size(); i++) { - RowExpression expression = lambdaAndTryExpressions.get(i); - - if (expression instanceof CallExpression) { - CallExpression tryExpression = (CallExpression) expression; - verify(!Signatures.TRY.equals(tryExpression.getSignature().getName())); - - Parameter session = arg("session", ConnectorSession.class); - Parameter cursor = arg("cursor", RecordCursor.class); - - List inputParameters = ImmutableList.builder() - .add(session) - .add(cursor) - .build(); - - BytecodeExpressionVisitor innerExpressionVisitor = new BytecodeExpressionVisitor( - callSiteBinder, - cachedInstanceBinder, - fieldReferenceCompiler(cursor), - metadata.getFunctionRegistry(), - new PreGeneratedExpressions(tryMethodMap.build(), lambdaFieldMap.build())); - - MethodDefinition tryMethod = defineTryMethod( - innerExpressionVisitor, - containerClassDefinition, - methodPrefix + "_try_" + i, - inputParameters, - Primitives.wrap(tryExpression.getType().getJavaType()), - tryExpression, - callSiteBinder); - - tryMethodMap.put(tryExpression, tryMethod); - } - else if (expression instanceof LambdaDefinitionExpression) { - LambdaDefinitionExpression lambdaExpression = (LambdaDefinitionExpression) expression; - String fieldName = methodPrefix + "_lambda_" + i; - PreGeneratedExpressions preGeneratedExpressions = new PreGeneratedExpressions(tryMethodMap.build(), lambdaFieldMap.build()); - FieldDefinition methodHandleField = LambdaBytecodeGenerator.preGenerateLambdaExpression( - lambdaExpression, - fieldName, - containerClassDefinition, - preGeneratedExpressions, - callSiteBinder, - cachedInstanceBinder, - metadata.getFunctionRegistry()); - lambdaFieldMap.put(lambdaExpression, methodHandleField); - } - else { - throw new VerifyException(format("unexpected expression: %s", expression.toString())); - } + + int methodId = 0; + for (CallExpression tryExpression : tryExpressions) { + Parameter session = arg("session", ConnectorSession.class); + Parameter cursor = arg("cursor", RecordCursor.class); + Parameter wasNull = arg("wasNull", boolean.class); + + List inputParameters = ImmutableList.builder() + .add(session) + .add(cursor) + .add(wasNull) + .build(); + + BytecodeExpressionVisitor innerExpressionVisitor = new BytecodeExpressionVisitor( + callSiteBinder, + cachedInstanceBinder, + fieldReferenceCompiler(cursor, wasNull), + metadata.getFunctionRegistry(), + tryMethodMap.build()); + + MethodDefinition tryMethod = defineTryMethod( + innerExpressionVisitor, + containerClassDefinition, + methodPrefix + "_try_" + methodId, + inputParameters, + Primitives.wrap(tryExpression.getType().getJavaType()), + tryExpression, + callSiteBinder); + + tryMethodMap.put(tryExpression, tryMethod); + methodId++; } - return new PreGeneratedExpressions(tryMethodMap.build(), lambdaFieldMap.build()); + return tryMethodMap.build(); } private void generateFilterMethod(ClassDefinition classDefinition, CallSiteBinder callSiteBinder, CachedInstanceBinder cachedInstanceBinder, RowExpression filter) { - PreGeneratedExpressions preGeneratedExpressions = generateMethodsForLambdaAndTry(classDefinition, callSiteBinder, cachedInstanceBinder, filter, "filter"); + Map tryMethodMap = generateTryMethods(classDefinition, callSiteBinder, cachedInstanceBinder, filter, "filter"); Parameter session = arg("session", ConnectorSession.class); Parameter cursor = arg("cursor", RecordCursor.class); @@ -263,9 +238,9 @@ private void generateFilterMethod(ClassDefinition classDefinition, CallSiteBinde BytecodeExpressionVisitor visitor = new BytecodeExpressionVisitor( callSiteBinder, cachedInstanceBinder, - fieldReferenceCompiler(cursor), + fieldReferenceCompiler(cursor, wasNullVariable), metadata.getFunctionRegistry(), - preGeneratedExpressions); + tryMethodMap); LabelNode end = new LabelNode("end"); method.getBody() @@ -284,7 +259,7 @@ private void generateFilterMethod(ClassDefinition classDefinition, CallSiteBinde private void generateProjectMethod(ClassDefinition classDefinition, CallSiteBinder callSiteBinder, CachedInstanceBinder cachedInstanceBinder, String methodName, RowExpression projection) { - PreGeneratedExpressions preGeneratedExpressions = generateMethodsForLambdaAndTry(classDefinition, callSiteBinder, cachedInstanceBinder, projection, methodName); + Map tryMethodMap = generateTryMethods(classDefinition, callSiteBinder, cachedInstanceBinder, projection, methodName); Parameter session = arg("session", ConnectorSession.class); Parameter cursor = arg("cursor", RecordCursor.class); @@ -296,24 +271,25 @@ private void generateProjectMethod(ClassDefinition classDefinition, CallSiteBind Scope scope = method.getScope(); Variable wasNullVariable = scope.declareVariable(type(boolean.class), "wasNull"); + BytecodeBlock body = method.getBody() + .comment("boolean wasNull = false;") + .putVariable(wasNullVariable, false); + BytecodeExpressionVisitor visitor = new BytecodeExpressionVisitor( callSiteBinder, cachedInstanceBinder, - fieldReferenceCompiler(cursor), + fieldReferenceCompiler(cursor, wasNullVariable), metadata.getFunctionRegistry(), - preGeneratedExpressions); + tryMethodMap); - method.getBody() - .comment("boolean wasNull = false;") - .putVariable(wasNullVariable, false) - .getVariable(output) + body.getVariable(output) .comment("evaluate projection: " + projection.toString()) .append(projection.accept(visitor, scope)) .append(generateWrite(callSiteBinder, scope, wasNullVariable, projection.getType())) .ret(); } - private static RowExpressionVisitor fieldReferenceCompiler(Variable cursorVariable) + private RowExpressionVisitor fieldReferenceCompiler(final Variable cursorVariable, final Variable wasNullVariable) { return new RowExpressionVisitor() { @@ -322,7 +298,6 @@ public BytecodeNode visitInputReference(InputReferenceExpression node, Scope sco { int field = node.getField(); Type type = node.getType(); - Variable wasNullVariable = scope.getVariable("wasNull"); Class javaType = type.getJavaType(); if (!javaType.isPrimitive() && javaType != Slice.class) { @@ -359,18 +334,6 @@ public BytecodeNode visitConstant(ConstantExpression literal, Scope scope) { throw new UnsupportedOperationException("not yet implemented"); } - - @Override - public BytecodeNode visitLambda(LambdaDefinitionExpression lambda, Scope context) - { - throw new UnsupportedOperationException(); - } - - @Override - public BytecodeNode visitVariableReference(VariableReferenceExpression reference, Scope context) - { - throw new UnsupportedOperationException(); - } }; } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/InCodeGenerator.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/InCodeGenerator.java index d7eec076f770..1563d1cd194a 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/InCodeGenerator.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/gen/InCodeGenerator.java @@ -271,7 +271,7 @@ public static boolean isInteger(long value) return value == (int) value; } - private static BytecodeBlock buildInCase(BytecodeGeneratorContext generatorContext, + private BytecodeBlock buildInCase(BytecodeGeneratorContext generatorContext, Scope scope, Type type, LabelNode caseLabel, diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/InputReferenceCompiler.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/InputReferenceCompiler.java index 8ee458b3caa4..b2b005286dbe 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/InputReferenceCompiler.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/gen/InputReferenceCompiler.java @@ -22,9 +22,7 @@ import com.facebook.presto.sql.relational.CallExpression; import com.facebook.presto.sql.relational.ConstantExpression; import com.facebook.presto.sql.relational.InputReferenceExpression; -import com.facebook.presto.sql.relational.LambdaDefinitionExpression; import com.facebook.presto.sql.relational.RowExpressionVisitor; -import com.facebook.presto.sql.relational.VariableReferenceExpression; import com.google.common.primitives.Primitives; import io.airlift.slice.Slice; @@ -38,15 +36,18 @@ class InputReferenceCompiler { private final BiFunction blockResolver; private final BiFunction positionResolver; + private final Variable wasNullVariable; private final CallSiteBinder callSiteBinder; public InputReferenceCompiler( BiFunction blockResolver, BiFunction positionResolver, + Variable wasNullVariable, CallSiteBinder callSiteBinder) { this.blockResolver = requireNonNull(blockResolver, "blockResolver is null"); this.positionResolver = requireNonNull(positionResolver, "positionResolver is null"); + this.wasNullVariable = requireNonNull(wasNullVariable, "wasNullVariable is null"); this.callSiteBinder = requireNonNull(callSiteBinder, "callSiteBinder is null"); } @@ -58,7 +59,6 @@ public BytecodeNode visitInputReference(InputReferenceExpression node, Scope sco BytecodeExpression block = blockResolver.apply(scope, field); BytecodeExpression position = positionResolver.apply(scope, field); - Variable wasNullVariable = scope.getVariable("wasNull"); Class javaType = type.getJavaType(); if (!javaType.isPrimitive() && javaType != Slice.class) { @@ -89,16 +89,4 @@ public BytecodeNode visitConstant(ConstantExpression literal, Scope scope) { throw new UnsupportedOperationException("not yet implemented"); } - - @Override - public BytecodeNode visitLambda(LambdaDefinitionExpression lambda, Scope context) - { - throw new UnsupportedOperationException(); - } - - @Override - public BytecodeNode visitVariableReference(VariableReferenceExpression reference, Scope context) - { - throw new UnsupportedOperationException(); - } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/JoinCompiler.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/JoinCompiler.java index fde6cde4d74f..c2fe9d22aa96 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/JoinCompiler.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/gen/JoinCompiler.java @@ -307,7 +307,7 @@ private static void generateAppendToMethod(ClassDefinition classDefinition, Call appendToBody.ret(); } - private static void generateIsPositionNull(ClassDefinition classDefinition, List joinChannelFields) + private void generateIsPositionNull(ClassDefinition classDefinition, List joinChannelFields) { Parameter blockIndex = arg("blockIndex", int.class); Parameter blockPosition = arg("blockPosition", int.class); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/JoinFilterFunctionCompiler.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/JoinFilterFunctionCompiler.java index b77ad3b7a64b..92e3217fdacc 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/JoinFilterFunctionCompiler.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/gen/JoinFilterFunctionCompiler.java @@ -30,12 +30,9 @@ import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.block.Block; import com.facebook.presto.sql.relational.CallExpression; -import com.facebook.presto.sql.relational.LambdaDefinitionExpression; import com.facebook.presto.sql.relational.RowExpression; import com.facebook.presto.sql.relational.RowExpressionVisitor; -import com.facebook.presto.sql.relational.Signatures; import com.google.common.base.Throwables; -import com.google.common.base.VerifyException; import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; @@ -47,6 +44,7 @@ import java.lang.reflect.Constructor; import java.util.List; +import java.util.Map; import java.util.Objects; import static com.facebook.presto.bytecode.Access.FINAL; @@ -59,11 +57,8 @@ import static com.facebook.presto.bytecode.ParameterizedType.type; import static com.facebook.presto.bytecode.expression.BytecodeExpressions.constantFalse; import static com.facebook.presto.sql.gen.BytecodeUtils.invoke; -import static com.facebook.presto.sql.gen.LambdaAndTryExpressionExtractor.extractLambdaAndTryExpressions; import static com.facebook.presto.sql.gen.TryCodeGenerator.defineTryMethod; import static com.google.common.base.MoreObjects.toStringHelper; -import static com.google.common.base.Verify.verify; -import static java.lang.String.format; import static java.util.Objects.requireNonNull; public class JoinFilterFunctionCompiler @@ -133,7 +128,7 @@ private void generateMethods(ClassDefinition classDefinition, CallSiteBinder cal generateConstructor(classDefinition, sessionField, cachedInstanceBinder); } - private static void generateConstructor(ClassDefinition classDefinition, FieldDefinition sessionField, CachedInstanceBinder cachedInstanceBinder) + private void generateConstructor(ClassDefinition classDefinition, FieldDefinition sessionField, CachedInstanceBinder cachedInstanceBinder) { Parameter sessionParameter = arg("session", ConnectorSession.class); MethodDefinition constructorDefinition = classDefinition.declareConstructor(a(PUBLIC), sessionParameter); @@ -154,7 +149,7 @@ private static void generateConstructor(ClassDefinition classDefinition, FieldDe private void generateFilterMethod(ClassDefinition classDefinition, CallSiteBinder callSiteBinder, CachedInstanceBinder cachedInstanceBinder, RowExpression filter, int leftBlocksSize, FieldDefinition sessionField) { - PreGeneratedExpressions preGeneratedExpressions = generateMethodsForLambdaAndTry(classDefinition, callSiteBinder, cachedInstanceBinder, leftBlocksSize, filter); + Map tryMethodMap = generateTryMethods(classDefinition, callSiteBinder, cachedInstanceBinder, leftBlocksSize, filter); // int leftPosition, Block[] leftBlocks, int rightPosition, Block[] rightBlocks Parameter leftPosition = arg("leftPosition", int.class); @@ -183,9 +178,9 @@ private void generateFilterMethod(ClassDefinition classDefinition, CallSiteBinde BytecodeExpressionVisitor visitor = new BytecodeExpressionVisitor( callSiteBinder, cachedInstanceBinder, - fieldReferenceCompiler(callSiteBinder, leftPosition, leftBlocks, rightPosition, rightBlocks, leftBlocksSize), + fieldReferenceCompiler(callSiteBinder, leftPosition, leftBlocks, rightPosition, rightBlocks, leftBlocksSize, wasNullVariable), metadata.getFunctionRegistry(), - preGeneratedExpressions); + tryMethodMap); BytecodeNode visitorBody = filter.accept(visitor, scope); @@ -198,75 +193,58 @@ private void generateFilterMethod(ClassDefinition classDefinition, CallSiteBinde .ifFalse(result.ret())); } - private PreGeneratedExpressions generateMethodsForLambdaAndTry( + private Map generateTryMethods( ClassDefinition containerClassDefinition, CallSiteBinder callSiteBinder, CachedInstanceBinder cachedInstanceBinder, int leftBlocksSize, RowExpression filter) { - List lambdaAndTryExpressions = extractLambdaAndTryExpressions(filter); + TryExpressionExtractor tryExtractor = new TryExpressionExtractor(); + filter.accept(tryExtractor, null); + List tryExpressions = tryExtractor.getTryExpressionsPreOrder(); + ImmutableMap.Builder tryMethodMap = ImmutableMap.builder(); - ImmutableMap.Builder lambdaFieldMap = ImmutableMap.builder(); - - for (int i = 0; i < lambdaAndTryExpressions.size(); i++) { - RowExpression expression = lambdaAndTryExpressions.get(i); - - if (expression instanceof CallExpression) { - CallExpression tryExpression = (CallExpression) expression; - verify(!Signatures.TRY.equals(tryExpression.getSignature().getName())); - - Parameter session = arg("session", ConnectorSession.class); - Parameter leftPosition = arg("leftPosition", int.class); - Parameter leftBlocks = arg("leftBlocks", Block[].class); - Parameter rightPosition = arg("rightPosition", int.class); - Parameter rightBlocks = arg("rightBlocks", Block[].class); - - BytecodeExpressionVisitor innerExpressionVisitor = new BytecodeExpressionVisitor( - callSiteBinder, - cachedInstanceBinder, - fieldReferenceCompiler(callSiteBinder, leftPosition, leftBlocks, rightPosition, rightBlocks, leftBlocksSize), - metadata.getFunctionRegistry(), - new PreGeneratedExpressions(tryMethodMap.build(), lambdaFieldMap.build())); - - List inputParameters = ImmutableList.builder() - .add(session) - .add(leftPosition) - .add(leftBlocks) - .add(rightPosition) - .add(rightBlocks) - .build(); - - MethodDefinition tryMethod = defineTryMethod( - innerExpressionVisitor, - containerClassDefinition, - "try_" + i, - inputParameters, - Primitives.wrap(tryExpression.getType().getJavaType()), - tryExpression, - callSiteBinder); - - tryMethodMap.put(tryExpression, tryMethod); - } - else if (expression instanceof LambdaDefinitionExpression) { - LambdaDefinitionExpression lambdaExpression = (LambdaDefinitionExpression) expression; - PreGeneratedExpressions preGeneratedExpressions = new PreGeneratedExpressions(tryMethodMap.build(), lambdaFieldMap.build()); - FieldDefinition methodHandleField = LambdaBytecodeGenerator.preGenerateLambdaExpression( - lambdaExpression, - "lambda_" + i, - containerClassDefinition, - preGeneratedExpressions, - callSiteBinder, - cachedInstanceBinder, - metadata.getFunctionRegistry()); - lambdaFieldMap.put(lambdaExpression, methodHandleField); - } - else { - throw new VerifyException(format("unexpected expression: %s", expression.toString())); - } + + int methodId = 0; + for (CallExpression tryExpression : tryExpressions) { + Parameter session = arg("session", ConnectorSession.class); + Parameter leftPosition = arg("leftPosition", int.class); + Parameter leftBlocks = arg("leftBlocks", Block[].class); + Parameter rightPosition = arg("rightPosition", int.class); + Parameter rightBlocks = arg("rightBlocks", Block[].class); + Parameter wasNullVariable = arg("wasNull", boolean.class); + + BytecodeExpressionVisitor innerExpressionVisitor = new BytecodeExpressionVisitor( + callSiteBinder, + cachedInstanceBinder, + fieldReferenceCompiler(callSiteBinder, leftPosition, leftBlocks, rightPosition, rightBlocks, leftBlocksSize, wasNullVariable), + metadata.getFunctionRegistry(), + tryMethodMap.build()); + + List inputParameters = ImmutableList.builder() + .add(session) + .add(leftPosition) + .add(leftBlocks) + .add(rightPosition) + .add(rightBlocks) + .add(wasNullVariable) + .build(); + + MethodDefinition tryMethod = defineTryMethod( + innerExpressionVisitor, + containerClassDefinition, + "try_" + methodId, + inputParameters, + Primitives.wrap(tryExpression.getType().getJavaType()), + tryExpression, + callSiteBinder); + + tryMethodMap.put(tryExpression, tryMethod); + methodId++; } - return new PreGeneratedExpressions(tryMethodMap.build(), lambdaFieldMap.build()); + return tryMethodMap.build(); } private static void generateToString(ClassDefinition classDefinition, CallSiteBinder callSiteBinder, String string) @@ -290,11 +268,13 @@ private static RowExpressionVisitor fieldReferenceCompiler( final Variable leftBlocks, final Variable rightPosition, final Variable rightBlocks, - final int leftBlocksSize) + final int leftBlocksSize, + final Variable wasNullVariable) { return new InputReferenceCompiler( (scope, field) -> field < leftBlocksSize ? leftBlocks.getElement(field) : rightBlocks.getElement(field - leftBlocksSize), (scope, field) -> field < leftBlocksSize ? leftPosition : rightPosition, + wasNullVariable, callSiteBinder); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/JoinProbeCompiler.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/JoinProbeCompiler.java index 13be54f4a95c..52709566c828 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/JoinProbeCompiler.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/gen/JoinProbeCompiler.java @@ -197,7 +197,7 @@ private Class compileJoinProbe(List types, List probeChannels, Optional probeHashChannel, FieldDefinition lookupSourceField, @@ -278,7 +278,7 @@ private static void generateConstructor(ClassDefinition classDefinition, constructor.ret(); } - private static void generateGetChannelCountMethod(ClassDefinition classDefinition, int channelCount) + private void generateGetChannelCountMethod(ClassDefinition classDefinition, int channelCount) { classDefinition.declareMethod( a(PUBLIC), @@ -289,7 +289,7 @@ private static void generateGetChannelCountMethod(ClassDefinition classDefinitio .retInt(); } - private static void generateAppendToMethod( + private void generateAppendToMethod( ClassDefinition classDefinition, CallSiteBinder callSiteBinder, List types, List blockFields, @@ -316,7 +316,7 @@ private static void generateAppendToMethod( .ret(); } - private static void generateAdvanceNextPosition(ClassDefinition classDefinition, FieldDefinition positionField, FieldDefinition positionCountField) + private void generateAdvanceNextPosition(ClassDefinition classDefinition, FieldDefinition positionField, FieldDefinition positionCountField) { MethodDefinition method = classDefinition.declareMethod( a(PUBLIC), @@ -350,7 +350,7 @@ private static void generateAdvanceNextPosition(ClassDefinition classDefinition, .retBoolean(); } - private static void generateGetCurrentJoinPosition(ClassDefinition classDefinition, + private void generateGetCurrentJoinPosition(ClassDefinition classDefinition, CallSiteBinder callSiteBinder, FieldDefinition lookupSourceField, FieldDefinition probePageField, @@ -390,7 +390,7 @@ private static void generateGetCurrentJoinPosition(ClassDefinition classDefiniti } } - private static void generateCurrentRowContainsNull(ClassDefinition classDefinition, List probeBlockFields, FieldDefinition positionField) + private void generateCurrentRowContainsNull(ClassDefinition classDefinition, List probeBlockFields, FieldDefinition positionField) { MethodDefinition method = classDefinition.declareMethod( a(PRIVATE), @@ -413,7 +413,7 @@ private static void generateCurrentRowContainsNull(ClassDefinition classDefiniti .retInt(); } - private static void generateGetPosition(ClassDefinition classDefinition, FieldDefinition positionField) + private void generateGetPosition(ClassDefinition classDefinition, FieldDefinition positionField) { // dummy implementation for now // compiled class is used only in usecase case when result of this method is ignored. @@ -427,7 +427,7 @@ private static void generateGetPosition(ClassDefinition classDefinition, FieldDe .retInt(); } - private static void generateGetPage(ClassDefinition classDefinition, FieldDefinition pageField) + private void generateGetPage(ClassDefinition classDefinition, FieldDefinition pageField) { // dummy implementation for now // compiled class is used only in usecase case when result of this method is ignored. diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/LambdaAndTryExpressionExtractor.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/LambdaAndTryExpressionExtractor.java deleted file mode 100644 index 0bf1ea3c7817..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/LambdaAndTryExpressionExtractor.java +++ /dev/null @@ -1,120 +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.sql.gen; - -import com.facebook.presto.spi.PrestoException; -import com.facebook.presto.spi.StandardErrorCode; -import com.facebook.presto.sql.relational.CallExpression; -import com.facebook.presto.sql.relational.ConstantExpression; -import com.facebook.presto.sql.relational.InputReferenceExpression; -import com.facebook.presto.sql.relational.LambdaDefinitionExpression; -import com.facebook.presto.sql.relational.RowExpression; -import com.facebook.presto.sql.relational.RowExpressionVisitor; -import com.facebook.presto.sql.relational.VariableReferenceExpression; -import com.google.common.collect.ImmutableList; - -import java.util.List; - -import static com.facebook.presto.sql.relational.Signatures.TRY; -import static com.google.common.base.Preconditions.checkState; -import static com.google.common.collect.Iterables.getOnlyElement; - -public class LambdaAndTryExpressionExtractor -{ - private LambdaAndTryExpressionExtractor() - { - } - - public static List extractLambdaAndTryExpressions(RowExpression expression) - { - Visitor visitor = new Visitor(); - expression.accept(visitor, new Context(false)); - return visitor.getLambdaAndTryExpressionsPostOrder(); - } - - private static class Visitor - implements RowExpressionVisitor - { - private final ImmutableList.Builder lambdaAndTryExpressions = ImmutableList.builder(); - - @Override - public Void visitInputReference(InputReferenceExpression node, Context context) - { - // TODO: change such that CallExpressions only capture the inputs they actually depend on - return null; - } - - @Override - public Void visitCall(CallExpression call, Context context) - { - boolean isTry = call.getSignature().getName().equals(TRY); - if (isTry) { - checkState(call.getArguments().size() == 1, "try call expressions must have a single argument"); - checkState(getOnlyElement(call.getArguments()) instanceof CallExpression, "try call expression argument must be a call expression"); - if (context.isInLambda()) { - throw new PrestoException(StandardErrorCode.NOT_SUPPORTED, "Try expression inside lambda expression is not support yet"); - } - } - - for (RowExpression rowExpression : call.getArguments()) { - rowExpression.accept(this, context); - } - - if (isTry) { - lambdaAndTryExpressions.add(getOnlyElement(call.getArguments())); - } - return null; - } - - @Override - public Void visitConstant(ConstantExpression literal, Context context) - { - return null; - } - - @Override - public Void visitLambda(LambdaDefinitionExpression lambda, Context context) - { - lambda.getBody().accept(this, new Context(true)); - lambdaAndTryExpressions.add(lambda); - return null; - } - - @Override - public Void visitVariableReference(VariableReferenceExpression reference, Context context) - { - return null; - } - - private List getLambdaAndTryExpressionsPostOrder() - { - return lambdaAndTryExpressions.build(); - } - } - - private static class Context - { - private final boolean inLambda; - - public Context(boolean inLambda) - { - this.inLambda = inLambda; - } - - public boolean isInLambda() - { - return inLambda; - } - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/LambdaBytecodeGenerator.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/LambdaBytecodeGenerator.java deleted file mode 100644 index c80498bea038..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/LambdaBytecodeGenerator.java +++ /dev/null @@ -1,183 +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.sql.gen; - -import com.facebook.presto.bytecode.BytecodeBlock; -import com.facebook.presto.bytecode.BytecodeNode; -import com.facebook.presto.bytecode.ClassDefinition; -import com.facebook.presto.bytecode.FieldDefinition; -import com.facebook.presto.bytecode.MethodDefinition; -import com.facebook.presto.bytecode.Parameter; -import com.facebook.presto.bytecode.Scope; -import com.facebook.presto.bytecode.Variable; -import com.facebook.presto.bytecode.expression.BytecodeExpressions; -import com.facebook.presto.metadata.FunctionRegistry; -import com.facebook.presto.spi.ConnectorSession; -import com.facebook.presto.sql.relational.CallExpression; -import com.facebook.presto.sql.relational.ConstantExpression; -import com.facebook.presto.sql.relational.InputReferenceExpression; -import com.facebook.presto.sql.relational.LambdaDefinitionExpression; -import com.facebook.presto.sql.relational.RowExpressionVisitor; -import com.facebook.presto.sql.relational.VariableReferenceExpression; -import com.facebook.presto.util.Reflection; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.primitives.Primitives; - -import java.lang.invoke.MethodHandle; -import java.util.List; -import java.util.Map; - -import static com.facebook.presto.bytecode.Access.FINAL; -import static com.facebook.presto.bytecode.Access.PRIVATE; -import static com.facebook.presto.bytecode.Access.PUBLIC; -import static com.facebook.presto.bytecode.Access.STATIC; -import static com.facebook.presto.bytecode.Access.a; -import static com.facebook.presto.bytecode.Parameter.arg; -import static com.facebook.presto.bytecode.ParameterizedType.type; -import static com.facebook.presto.bytecode.expression.BytecodeExpressions.constantClass; -import static com.facebook.presto.bytecode.expression.BytecodeExpressions.constantString; -import static com.facebook.presto.bytecode.expression.BytecodeExpressions.invokeStatic; -import static com.facebook.presto.bytecode.expression.BytecodeExpressions.newArray; -import static com.facebook.presto.bytecode.expression.BytecodeExpressions.setStatic; -import static com.facebook.presto.sql.gen.BytecodeUtils.boxPrimitiveIfNecessary; -import static com.facebook.presto.sql.gen.BytecodeUtils.unboxPrimitiveIfNecessary; -import static com.facebook.presto.util.ImmutableCollectors.toImmutableList; - -public class LambdaBytecodeGenerator -{ - private LambdaBytecodeGenerator() - { - } - - /** - * @return a MethodHandle field that represents the lambda expression - */ - public static FieldDefinition preGenerateLambdaExpression( - LambdaDefinitionExpression lambdaExpression, - String fieldName, - ClassDefinition classDefinition, - PreGeneratedExpressions preGeneratedExpressions, - CallSiteBinder callSiteBinder, - CachedInstanceBinder cachedInstanceBinder, - FunctionRegistry functionRegistry) - { - ImmutableList.Builder parameters = ImmutableList.builder(); - ImmutableMap.Builder parameterMapBuilder = ImmutableMap.builder(); - - parameters.add(arg("session", ConnectorSession.class)); - for (int i = 0; i < lambdaExpression.getArguments().size(); i++) { - Class type = Primitives.wrap(lambdaExpression.getArgumentTypes().get(i).getJavaType()); - String argumentName = lambdaExpression.getArguments().get(i); - Parameter arg = arg("lambda_" + argumentName, type); - parameters.add(arg); - parameterMapBuilder.put(argumentName, new ParameterAndType(arg, type)); - } - - BytecodeExpressionVisitor innerExpressionVisitor = new BytecodeExpressionVisitor( - callSiteBinder, - cachedInstanceBinder, - variableReferenceCompiler(parameterMapBuilder.build()), - functionRegistry, - preGeneratedExpressions); - - return defineLambdaMethodAndField( - innerExpressionVisitor, - classDefinition, - fieldName, - parameters.build(), - lambdaExpression); - } - - private static FieldDefinition defineLambdaMethodAndField( - BytecodeExpressionVisitor innerExpressionVisitor, - ClassDefinition classDefinition, - String fieldAndMethodName, - List inputParameters, - LambdaDefinitionExpression lambda) - { - Class returnType = Primitives.wrap(lambda.getBody().getType().getJavaType()); - MethodDefinition method = classDefinition.declareMethod(a(PUBLIC), fieldAndMethodName, type(returnType), inputParameters); - - Scope scope = method.getScope(); - Variable wasNull = scope.declareVariable(boolean.class, "wasNull"); - BytecodeNode compiledBody = lambda.getBody().accept(innerExpressionVisitor, scope); - method.getBody() - .putVariable(wasNull, false) - .append(compiledBody) - .append(boxPrimitiveIfNecessary(scope, returnType)) - .ret(returnType); - - FieldDefinition methodHandleField = classDefinition.declareField(a(PRIVATE, STATIC, FINAL), fieldAndMethodName, type(MethodHandle.class)); - - classDefinition.getClassInitializer().getBody() - .append(setStatic( - methodHandleField, - invokeStatic( - Reflection.class, - "methodHandle", - MethodHandle.class, - constantClass(classDefinition.getType()), - constantString(fieldAndMethodName), - newArray( - type(Class[].class), - inputParameters.stream() - .map(Parameter::getType) - .map(BytecodeExpressions::constantClass) - .collect(toImmutableList()))))); - return methodHandleField; - } - - private static RowExpressionVisitor variableReferenceCompiler(Map parameterMap) - { - return new RowExpressionVisitor() - { - @Override - public BytecodeNode visitInputReference(InputReferenceExpression node, Scope scope) - { - throw new UnsupportedOperationException(); - } - - @Override - public BytecodeNode visitCall(CallExpression call, Scope scope) - { - throw new UnsupportedOperationException(); - } - - @Override - public BytecodeNode visitConstant(ConstantExpression literal, Scope scope) - { - throw new UnsupportedOperationException(); - } - - @Override - public BytecodeNode visitLambda(LambdaDefinitionExpression lambda, Scope context) - { - throw new UnsupportedOperationException(); - } - - @Override - public BytecodeNode visitVariableReference(VariableReferenceExpression reference, Scope context) - { - ParameterAndType parameterAndType = parameterMap.get(reference.getName()); - Parameter parameter = parameterAndType.getParameter(); - Class type = parameterAndType.getType(); - return new BytecodeBlock() - .append(parameter) - .append(unboxPrimitiveIfNecessary(context, type)); - } - }; - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/NullIfCodeGenerator.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/NullIfCodeGenerator.java index dfe495463fb3..1b15720f2dd8 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/NullIfCodeGenerator.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/gen/NullIfCodeGenerator.java @@ -81,7 +81,7 @@ public BytecodeNode generateExpression(Signature signature, BytecodeGeneratorCon return block; } - private static BytecodeNode cast( + private BytecodeNode cast( BytecodeGeneratorContext generatorContext, BytecodeNode argument, Type actualType, diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/OrderingCompiler.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/OrderingCompiler.java index 8ad194334a4b..473fe284453b 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/OrderingCompiler.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/gen/OrderingCompiler.java @@ -127,7 +127,7 @@ private Class compilePagesIndexComparator( return defineClass(classDefinition, PagesIndexComparator.class, callSiteBinder.getBindings(), getClass().getClassLoader()); } - private static void generateCompareTo(ClassDefinition classDefinition, CallSiteBinder callSiteBinder, List sortTypes, List sortChannels, List sortOrders) + private void generateCompareTo(ClassDefinition classDefinition, CallSiteBinder callSiteBinder, List sortTypes, List sortChannels, List sortOrders) { Parameter pagesIndex = arg("pagesIndex", PagesIndex.class); Parameter leftPosition = arg("leftPosition", int.class); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/PageProcessorCompiler.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/PageProcessorCompiler.java index c7fa539651c2..3abf3f40887f 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/PageProcessorCompiler.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/gen/PageProcessorCompiler.java @@ -42,11 +42,8 @@ import com.facebook.presto.sql.relational.DeterminismEvaluator; import com.facebook.presto.sql.relational.Expressions; import com.facebook.presto.sql.relational.InputReferenceExpression; -import com.facebook.presto.sql.relational.LambdaDefinitionExpression; import com.facebook.presto.sql.relational.RowExpression; import com.facebook.presto.sql.relational.RowExpressionVisitor; -import com.facebook.presto.sql.relational.Signatures; -import com.google.common.base.VerifyException; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.primitives.Primitives; @@ -79,13 +76,11 @@ import static com.facebook.presto.bytecode.instruction.JumpInstruction.jump; import static com.facebook.presto.sql.gen.BytecodeUtils.generateWrite; import static com.facebook.presto.sql.gen.BytecodeUtils.loadConstant; -import static com.facebook.presto.sql.gen.LambdaAndTryExpressionExtractor.extractLambdaAndTryExpressions; import static com.facebook.presto.sql.gen.TryCodeGenerator.defineTryMethod; import static com.google.common.base.Preconditions.checkState; import static com.google.common.base.Verify.verify; import static com.google.common.collect.Iterables.concat; import static com.google.common.collect.Iterables.getOnlyElement; -import static java.lang.String.format; import static java.util.Collections.singletonList; import static java.util.stream.Collectors.toList; @@ -285,7 +280,7 @@ private static MethodDefinition generateProjectColumnarMethod( Scope scope = method.getScope(); Variable thisVariable = method.getThis(); - ImmutableList.Builder builder = ImmutableList.builder(); + ImmutableList.Builder builder = ImmutableList.builder(); for (int channel : getInputChannels(projection)) { Variable blockVariable = scope.declareVariable("block_" + channel, body, page.invoke("getBlock", Block.class, constantInt(channel))); builder.add(blockVariable); @@ -632,7 +627,7 @@ private void generateFilterPageMethod(ClassDefinition classDefinition, RowExpres List filterChannels = getInputChannels(filter); // extract block variables - ImmutableList.Builder blockVariablesBuilder = ImmutableList.builder(); + ImmutableList.Builder blockVariablesBuilder = ImmutableList.builder(); for (int channel : filterChannels) { Variable blockVariable = scope.declareVariable("block_" + channel, body, page.invoke("getBlock", Block.class, constantInt(channel))); blockVariablesBuilder.add(blockVariable); @@ -736,76 +731,59 @@ private static BytecodeBlock getBytecodeFilterOnDictionary( return ifFilterOnDictionaryBlock; } - private PreGeneratedExpressions generateMethodsForLambdaAndTry( + private Map generateTryMethods( ClassDefinition containerClassDefinition, CallSiteBinder callSiteBinder, CachedInstanceBinder cachedInstanceBinder, RowExpression projection, String methodPrefix) { - List lambdaAndTryExpressions = extractLambdaAndTryExpressions(projection); - ImmutableMap.Builder tryMethodMap = ImmutableMap.builder(); - ImmutableMap.Builder lambdaFieldMap = ImmutableMap.builder(); - - for (int i = 0; i < lambdaAndTryExpressions.size(); i++) { - RowExpression expression = lambdaAndTryExpressions.get(i); - - if (expression instanceof CallExpression) { - CallExpression tryExpression = (CallExpression) expression; - verify(!Signatures.TRY.equals(tryExpression.getSignature().getName())); + TryExpressionExtractor tryExtractor = new TryExpressionExtractor(); + projection.accept(tryExtractor, null); + List tryExpressions = tryExtractor.getTryExpressionsPreOrder(); - Parameter session = arg("session", ConnectorSession.class); - List blocks = toBlockParameters(getInputChannels(tryExpression.getArguments())); - Parameter position = arg("position", int.class); + ImmutableMap.Builder tryMethodMap = ImmutableMap.builder(); - BytecodeExpressionVisitor innerExpressionVisitor = new BytecodeExpressionVisitor( - callSiteBinder, - cachedInstanceBinder, - fieldReferenceCompiler(callSiteBinder), - metadata.getFunctionRegistry(), - new PreGeneratedExpressions(tryMethodMap.build(), lambdaFieldMap.build())); + int methodId = 0; + for (CallExpression tryExpression : tryExpressions) { + Parameter session = arg("session", ConnectorSession.class); + List blocks = toBlockParameters(getInputChannels(tryExpression.getArguments())); + Parameter position = arg("position", int.class); + Parameter wasNullVariable = arg("wasNull", boolean.class); + + BytecodeExpressionVisitor innerExpressionVisitor = new BytecodeExpressionVisitor( + callSiteBinder, + cachedInstanceBinder, + fieldReferenceCompiler(callSiteBinder, position, wasNullVariable), + metadata.getFunctionRegistry(), + tryMethodMap.build()); + + List inputParameters = ImmutableList.builder() + .add(session) + .addAll(blocks) + .add(position) + .add(wasNullVariable) + .build(); - List inputParameters = ImmutableList.builder() - .add(session) - .addAll(blocks) - .add(position) - .build(); - - MethodDefinition tryMethod = defineTryMethod( - innerExpressionVisitor, - containerClassDefinition, - methodPrefix + "_try_" + i, - inputParameters, - Primitives.wrap(tryExpression.getType().getJavaType()), - tryExpression, - callSiteBinder); - - tryMethodMap.put(tryExpression, tryMethod); - } - else if (expression instanceof LambdaDefinitionExpression) { - LambdaDefinitionExpression lambdaExpression = (LambdaDefinitionExpression) expression; - PreGeneratedExpressions preGeneratedExpressions = new PreGeneratedExpressions(tryMethodMap.build(), lambdaFieldMap.build()); - FieldDefinition methodHandleField = LambdaBytecodeGenerator.preGenerateLambdaExpression( - lambdaExpression, - methodPrefix + "_lambda_" + i, - containerClassDefinition, - preGeneratedExpressions, - callSiteBinder, - cachedInstanceBinder, - metadata.getFunctionRegistry()); - lambdaFieldMap.put(lambdaExpression, methodHandleField); - } - else { - throw new VerifyException(format("unexpected expression: %s", expression.toString())); - } + MethodDefinition tryMethod = defineTryMethod( + innerExpressionVisitor, + containerClassDefinition, + methodPrefix + "_try_" + methodId, + inputParameters, + Primitives.wrap(tryExpression.getType().getJavaType()), + tryExpression, + callSiteBinder); + + tryMethodMap.put(tryExpression, tryMethod); + methodId++; } - return new PreGeneratedExpressions(tryMethodMap.build(), lambdaFieldMap.build()); + return tryMethodMap.build(); } private void generateFilterMethod(ClassDefinition classDefinition, CallSiteBinder callSiteBinder, CachedInstanceBinder cachedInstanceBinder, RowExpression filter) { - PreGeneratedExpressions preGeneratedExpressions = generateMethodsForLambdaAndTry(classDefinition, callSiteBinder, cachedInstanceBinder, filter, "filter"); + Map tryMethodMap = generateTryMethods(classDefinition, callSiteBinder, cachedInstanceBinder, filter, "filter"); Parameter session = arg("session", ConnectorSession.class); List blocks = toBlockParameters(getInputChannels(filter)); @@ -830,9 +808,9 @@ private void generateFilterMethod(ClassDefinition classDefinition, CallSiteBinde BytecodeExpressionVisitor visitor = new BytecodeExpressionVisitor( callSiteBinder, cachedInstanceBinder, - fieldReferenceCompiler(callSiteBinder), + fieldReferenceCompiler(callSiteBinder, position, wasNullVariable), metadata.getFunctionRegistry(), - preGeneratedExpressions); + tryMethodMap); BytecodeNode visitorBody = filter.accept(visitor, scope); @@ -847,7 +825,7 @@ private void generateFilterMethod(ClassDefinition classDefinition, CallSiteBinde private MethodDefinition generateProjectMethod(ClassDefinition classDefinition, CallSiteBinder callSiteBinder, CachedInstanceBinder cachedInstanceBinder, String methodName, RowExpression projection) { - PreGeneratedExpressions preGeneratedExpressions = generateMethodsForLambdaAndTry(classDefinition, callSiteBinder, cachedInstanceBinder, projection, methodName); + Map tryMethodMap = generateTryMethods(classDefinition, callSiteBinder, cachedInstanceBinder, projection, methodName); Parameter session = arg("session", ConnectorSession.class); List blocks = toBlockParameters(getInputChannels(projection)); @@ -874,9 +852,10 @@ private MethodDefinition generateProjectMethod(ClassDefinition classDefinition, BytecodeExpressionVisitor visitor = new BytecodeExpressionVisitor( callSiteBinder, cachedInstanceBinder, - fieldReferenceCompiler(callSiteBinder), + fieldReferenceCompiler(callSiteBinder, position, wasNullVariable), metadata.getFunctionRegistry(), - preGeneratedExpressions); + tryMethodMap + ); body.getVariable(output) .comment("evaluate projection: " + projection.toString()) @@ -925,11 +904,12 @@ private static List toBlockParameters(List inputChannels) return parameters.build(); } - private static RowExpressionVisitor fieldReferenceCompiler(CallSiteBinder callSiteBinder) + private static RowExpressionVisitor fieldReferenceCompiler(final CallSiteBinder callSiteBinder, final Variable positionVariable, final Variable wasNullVariable) { return new InputReferenceCompiler( (scope, field) -> scope.getVariable("block_" + field), - (scope, field) -> scope.getVariable("position"), + (scope, field) -> positionVariable, + wasNullVariable, callSiteBinder); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/ParameterAndType.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/ParameterAndType.java deleted file mode 100644 index 35ca3ce70e89..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/ParameterAndType.java +++ /dev/null @@ -1,41 +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.sql.gen; - -import com.facebook.presto.bytecode.Parameter; - -import static java.util.Objects.requireNonNull; - -class ParameterAndType -{ - private final Parameter parameter; - private final Class type; - - public ParameterAndType(Parameter parameter, Class type) - { - this.parameter = requireNonNull(parameter, "parameter is null"); - this.type = requireNonNull(type, "type is null"); - } - - public Parameter getParameter() - { - return parameter; - } - - public Class getType() - { - return type; - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/PreGeneratedExpressions.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/PreGeneratedExpressions.java deleted file mode 100644 index 2eedc6d50746..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/PreGeneratedExpressions.java +++ /dev/null @@ -1,47 +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.sql.gen; - -import com.facebook.presto.bytecode.FieldDefinition; -import com.facebook.presto.bytecode.MethodDefinition; -import com.facebook.presto.sql.relational.CallExpression; -import com.facebook.presto.sql.relational.LambdaDefinitionExpression; -import com.google.common.collect.ImmutableMap; - -import java.util.Map; - -import static java.util.Objects.requireNonNull; - -public class PreGeneratedExpressions -{ - private final Map tryMethodMap; - private final Map lambdaFieldMap; - - public PreGeneratedExpressions(Map tryMethodMap, Map lambdaFieldMap) - { - this.tryMethodMap = ImmutableMap.copyOf(requireNonNull(tryMethodMap, "tryMethodMap is null")); - this.lambdaFieldMap = ImmutableMap.copyOf(requireNonNull(lambdaFieldMap, "lambdaFieldMap is null")); - } - - public Map getTryMethodMap() - { - return tryMethodMap; - } - - public Map getLambdaFieldMap() - { - return lambdaFieldMap; - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/TryCodeGenerator.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/TryCodeGenerator.java index 85e1f9dd695a..f3cfb733c1ee 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/TryCodeGenerator.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/gen/TryCodeGenerator.java @@ -20,14 +20,12 @@ import com.facebook.presto.bytecode.Parameter; import com.facebook.presto.bytecode.ParameterizedType; import com.facebook.presto.bytecode.Scope; -import com.facebook.presto.bytecode.Variable; import com.facebook.presto.bytecode.control.TryCatch; import com.facebook.presto.metadata.Signature; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.type.Type; import com.facebook.presto.sql.relational.CallExpression; import com.facebook.presto.sql.relational.RowExpression; -import com.google.common.collect.ImmutableList; import com.google.common.primitives.Primitives; import java.lang.invoke.MethodHandle; @@ -38,7 +36,6 @@ import static com.facebook.presto.bytecode.Access.PUBLIC; import static com.facebook.presto.bytecode.Access.a; import static com.facebook.presto.bytecode.ParameterizedType.type; -import static com.facebook.presto.bytecode.expression.BytecodeExpressions.constantBoolean; import static com.facebook.presto.spi.StandardErrorCode.DIVISION_BY_ZERO; import static com.facebook.presto.spi.StandardErrorCode.INVALID_CAST_ARGUMENT; import static com.facebook.presto.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; @@ -46,7 +43,6 @@ import static com.facebook.presto.sql.gen.BytecodeUtils.boxPrimitiveIfNecessary; import static com.facebook.presto.sql.gen.BytecodeUtils.invoke; import static com.facebook.presto.sql.gen.BytecodeUtils.unboxPrimitiveIfNecessary; -import static com.facebook.presto.util.ImmutableCollectors.toImmutableList; import static com.facebook.presto.util.Reflection.methodHandle; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; @@ -74,15 +70,21 @@ public BytecodeNode generateExpression(Signature signature, BytecodeGeneratorCon CallExpression innerCallExpression = (CallExpression) getOnlyElement(arguments); checkState(tryMethodsMap.containsKey(innerCallExpression), "try methods map does not contain this try call"); + BytecodeBlock bytecodeBlock = new BytecodeBlock() + .comment("load required variables") + .getVariable(context.getScope().getVariable("this")); + MethodDefinition definition = tryMethodsMap.get(innerCallExpression); - ImmutableList invokeArguments = definition.getParameters().stream() + definition.getParameters().stream() .map(parameter -> context.getScope().getVariable(parameter.getName())) - .collect(toImmutableList()); + .forEach(bytecodeBlock::getVariable); - return new BytecodeBlock() - .append(context.getScope().getThis().invoke(definition, invokeArguments)) + bytecodeBlock.comment("call dynamic try method: " + definition.getName()) + .invokeVirtual(definition) .append(unboxPrimitiveIfNecessary(context.getScope(), Primitives.wrap(innerCallExpression.getType().getJavaType()))); + + return bytecodeBlock; } public static MethodDefinition defineTryMethod( @@ -97,7 +99,6 @@ public static MethodDefinition defineTryMethod( MethodDefinition method = classDefinition.declareMethod(a(PUBLIC), methodName, type(returnType), inputParameters); Scope calleeMethodScope = method.getScope(); - Variable wasNull = calleeMethodScope.declareVariable(boolean.class, "wasNull"); BytecodeNode innerExpression = innerRowExpression.accept(innerExpressionVisitor, calleeMethodScope); MethodType exceptionHandlerType = methodType(returnType, PrestoException.class); @@ -106,7 +107,6 @@ public static MethodDefinition defineTryMethod( method.comment("Try projection: %s", innerRowExpression.toString()); method.getBody() - .append(wasNull.set(constantBoolean(false))) .append(new TryCatch( new BytecodeBlock() .append(innerExpression) diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/TryExpressionExtractor.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/TryExpressionExtractor.java new file mode 100644 index 000000000000..c90c53b4a8dd --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/gen/TryExpressionExtractor.java @@ -0,0 +1,70 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.sql.gen; + +import com.facebook.presto.bytecode.BytecodeNode; +import com.facebook.presto.bytecode.Scope; +import com.facebook.presto.sql.relational.CallExpression; +import com.facebook.presto.sql.relational.ConstantExpression; +import com.facebook.presto.sql.relational.InputReferenceExpression; +import com.facebook.presto.sql.relational.RowExpression; +import com.facebook.presto.sql.relational.RowExpressionVisitor; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +import static com.facebook.presto.sql.relational.Signatures.TRY; +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.collect.Iterables.getOnlyElement; + +public class TryExpressionExtractor + implements RowExpressionVisitor +{ + private final ImmutableList.Builder tryExpressions = ImmutableList.builder(); + + @Override + public BytecodeNode visitInputReference(InputReferenceExpression node, Scope scope) + { + // TODO: change such that CallExpressions only capture the inputs they actually depend on + return null; + } + + @Override + public BytecodeNode visitCall(CallExpression call, Scope scope) + { + if (call.getSignature().getName().equals(TRY)) { + checkState(call.getArguments().size() == 1, "try call expressions must have a single argument"); + checkState(getOnlyElement(call.getArguments()) instanceof CallExpression, "try call expression argument must be a call expression"); + + tryExpressions.add((CallExpression) getOnlyElement(call.getArguments())); + } + + for (RowExpression rowExpression : call.getArguments()) { + rowExpression.accept(this, null); + } + + return null; + } + + @Override + public BytecodeNode visitConstant(ConstantExpression literal, Scope scope) + { + return null; + } + + public List getTryExpressionsPreOrder() + { + return tryExpressions.build().reverse(); + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/gen/VarArgsToMapAdapterGenerator.java b/presto-main/src/main/java/com/facebook/presto/sql/gen/VarArgsToMapAdapterGenerator.java deleted file mode 100644 index 2f15b9cbb643..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/sql/gen/VarArgsToMapAdapterGenerator.java +++ /dev/null @@ -1,93 +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.sql.gen; - -import com.facebook.presto.bytecode.BytecodeBlock; -import com.facebook.presto.bytecode.ClassDefinition; -import com.facebook.presto.bytecode.DynamicClassLoader; -import com.facebook.presto.bytecode.MethodDefinition; -import com.facebook.presto.bytecode.Parameter; -import com.facebook.presto.bytecode.ParameterizedType; -import com.facebook.presto.bytecode.Variable; -import com.facebook.presto.util.Reflection; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Maps; - -import java.lang.invoke.MethodHandle; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.function.Function; - -import static com.facebook.presto.bytecode.Access.FINAL; -import static com.facebook.presto.bytecode.Access.PUBLIC; -import static com.facebook.presto.bytecode.Access.STATIC; -import static com.facebook.presto.bytecode.Access.a; -import static com.facebook.presto.bytecode.CompilerUtils.defineClass; -import static com.facebook.presto.bytecode.CompilerUtils.makeClassName; -import static com.facebook.presto.bytecode.Parameter.arg; -import static com.facebook.presto.bytecode.ParameterizedType.type; -import static com.facebook.presto.bytecode.expression.BytecodeExpressions.constantInt; -import static com.facebook.presto.bytecode.expression.BytecodeExpressions.constantString; -import static com.facebook.presto.bytecode.expression.BytecodeExpressions.invokeStatic; -import static com.facebook.presto.sql.gen.BytecodeUtils.loadConstant; - -public class VarArgsToMapAdapterGenerator -{ - private VarArgsToMapAdapterGenerator() - { - } - - /** - * Generate byte code that - *

    - *
  • takes a specified number of variables as arguments (types of the arguments are provided in {@code javaTypes}) - *
  • put the variables in a map (keys of the map are provided in {@code names}) - *
  • invoke the provided {@code function} with the map - *
  • return with the result of the function call (type must match {@code returnType}) - *

- */ - public static MethodHandle generateVarArgsToMapAdapter(Class returnType, List> javaTypes, List names, Function, Object> function) - { - CallSiteBinder callSiteBinder = new CallSiteBinder(); - - ClassDefinition classDefinition = new ClassDefinition(a(PUBLIC, FINAL), makeClassName("VarArgsToMapAdapter"), type(Object.class)); - - ImmutableList.Builder parameterListBuilder = ImmutableList.builder(); - for (int i = 0; i < javaTypes.size(); i++) { - Class javaType = javaTypes.get(i); - parameterListBuilder.add(arg("input_" + i, javaType)); - } - ImmutableList parameterList = parameterListBuilder.build(); - - MethodDefinition methodDefinition = classDefinition.declareMethod(a(PUBLIC, STATIC), "varArgsToMap", ParameterizedType.type(returnType), parameterList); - BytecodeBlock body = methodDefinition.getBody(); - - // ImmutableMap.Builder can not be used here because it doesn't allow nulls. - Variable map = methodDefinition.getScope().declareVariable(HashMap.class, "map"); - body.append(map.set(invokeStatic(Maps.class, "newHashMapWithExpectedSize", HashMap.class, constantInt(javaTypes.size())))); - for (int i = 0; i < javaTypes.size(); i++) { - body.append(map.invoke("put", Object.class, constantString(names.get(i)).cast(Object.class), parameterList.get(i).cast(Object.class))); - } - body.append( - loadConstant(callSiteBinder, function, Function.class) - .invoke("apply", Object.class, map.cast(Object.class)) - .cast(returnType) - .ret()); - - Class generatedClass = defineClass(classDefinition, Object.class, callSiteBinder.getBindings(), new DynamicClassLoader(VarArgsToMapAdapterGenerator.class.getClassLoader())); - return Reflection.methodHandle(generatedClass, "varArgsToMap", javaTypes.toArray(new Class[javaTypes.size()])); - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/DomainTranslator.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/DomainTranslator.java index 583e74ce20ba..1cf08b609399 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/DomainTranslator.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/DomainTranslator.java @@ -429,7 +429,7 @@ private Optional coerce(NullableValue value, Type targetType) return Optional.of(NullableValue.of(targetType, coercedValue)); } - private static ExtractionResult createComparisonExtractionResult(ComparisonExpressionType comparisonType, Symbol column, Type type, @Nullable Object value, boolean complement) + private ExtractionResult createComparisonExtractionResult(ComparisonExpressionType comparisonType, Symbol column, Type type, @Nullable Object value, boolean complement) { if (value == null) { switch (comparisonType) { diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/EffectivePredicateExtractor.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/EffectivePredicateExtractor.java index e70f12e2ad2e..59e011e3650b 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/EffectivePredicateExtractor.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/EffectivePredicateExtractor.java @@ -260,7 +260,7 @@ public Expression visitJoin(JoinNode node, Void context) } } - private static Iterable pullNullableConjunctsThroughOuterJoin(List conjuncts, Predicate... nullSymbolScopes) + private Iterable pullNullableConjunctsThroughOuterJoin(List conjuncts, Predicate... nullSymbolScopes) { // Conjuncts without any symbol dependencies cannot be applied to the effective predicate (e.g. FALSE literal) return conjuncts.stream() diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/ExpressionInterpreter.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/ExpressionInterpreter.java index 4480e623492d..02b5ad0d32a7 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/ExpressionInterpreter.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/ExpressionInterpreter.java @@ -30,7 +30,6 @@ import com.facebook.presto.spi.function.OperatorType; 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.sql.analyzer.ExpressionAnalyzer; import com.facebook.presto.sql.analyzer.Scope; import com.facebook.presto.sql.analyzer.SemanticErrorCode; @@ -59,8 +58,6 @@ import com.facebook.presto.sql.tree.InPredicate; import com.facebook.presto.sql.tree.IsNotNullPredicate; import com.facebook.presto.sql.tree.IsNullPredicate; -import com.facebook.presto.sql.tree.LambdaArgumentDeclaration; -import com.facebook.presto.sql.tree.LambdaExpression; import com.facebook.presto.sql.tree.LikePredicate; import com.facebook.presto.sql.tree.Literal; import com.facebook.presto.sql.tree.LogicalBinaryExpression; @@ -82,7 +79,6 @@ import com.facebook.presto.sql.tree.TryExpression; import com.facebook.presto.sql.tree.WhenClause; import com.facebook.presto.type.ArrayType; -import com.facebook.presto.type.FunctionType; import com.facebook.presto.type.LikeFunctions; import com.facebook.presto.type.RowType; import com.facebook.presto.type.RowType.RowField; @@ -95,7 +91,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; -import com.google.common.primitives.Primitives; +import com.google.common.collect.Lists; import io.airlift.joni.Regex; import io.airlift.json.JsonCodec; import io.airlift.slice.Slice; @@ -104,7 +100,6 @@ import java.util.ArrayList; import java.util.IdentityHashMap; import java.util.List; -import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -115,9 +110,7 @@ import static com.facebook.presto.spi.type.TypeUtils.writeNativeValue; import static com.facebook.presto.sql.analyzer.ExpressionAnalyzer.createConstantAnalyzer; import static com.facebook.presto.sql.analyzer.SemanticErrorCode.EXPRESSION_NOT_CONSTANT; -import static com.facebook.presto.sql.analyzer.TypeSignatureProvider.fromTypes; import static com.facebook.presto.sql.gen.TryCodeGenerator.tryExpressionExceptionHandler; -import static com.facebook.presto.sql.gen.VarArgsToMapAdapterGenerator.generateVarArgsToMapAdapter; import static com.facebook.presto.sql.planner.LiteralInterpreter.toExpression; import static com.facebook.presto.sql.planner.LiteralInterpreter.toExpressions; import static com.facebook.presto.util.ImmutableCollectors.toImmutableList; @@ -946,7 +939,7 @@ protected Object visitFunctionCall(FunctionCall node, Object context) argumentValues.add(value); argumentTypes.add(type); } - Signature functionSignature = metadata.getFunctionRegistry().resolveFunction(node.getName(), fromTypes(argumentTypes)); + Signature functionSignature = metadata.getFunctionRegistry().resolveFunction(node.getName(), Lists.transform(argumentTypes, Type::getTypeSignature)); ScalarFunctionImplementation function = metadata.getFunctionRegistry().getScalarFunctionImplementation(functionSignature); for (int i = 0; i < argumentValues.size(); i++) { Object value = argumentValues.get(i); @@ -962,32 +955,6 @@ protected Object visitFunctionCall(FunctionCall node, Object context) return invoke(session, function, argumentValues); } - @Override - protected Object visitLambdaExpression(LambdaExpression node, Object context) - { - if (optimize) { - // TODO: enable optimization related to lambda expression - // A mechanism to convert function type back into lambda expression need to exist to enable optimization - return node; - } - - Expression body = node.getBody(); - List argumentNames = node.getArguments().stream() - .map(LambdaArgumentDeclaration::getName) - .collect(toImmutableList()); - FunctionType functionType = (FunctionType) expressionTypes.get(node); - checkArgument(argumentNames.size() == functionType.getArgumentTypes().size()); - - return generateVarArgsToMapAdapter( - Primitives.wrap(functionType.getReturnType().getJavaType()), - functionType.getArgumentTypes().stream() - .map(Type::getJavaType) - .map(Primitives::wrap) - .collect(toImmutableList()), - argumentNames, - map -> process(body, new LambdaSymbolResolver(map))); - } - @Override protected Object visitLikePredicate(LikePredicate node, Object context) { @@ -1037,21 +1004,9 @@ protected Object visitLikePredicate(LikePredicate node, Object context) if (pattern instanceof Slice && escape == null) { String stringPattern = ((Slice) pattern).toStringUtf8(); if (!stringPattern.contains("%") && !stringPattern.contains("_")) { - Type valueType = expressionTypes.get(node.getValue()); - Type patternType = expressionTypes.get(node.getPattern()); - TypeManager typeManager = metadata.getTypeManager(); - Optional commonSuperType = typeManager.getCommonSuperType(valueType, patternType); - checkArgument(commonSuperType.isPresent(), "Missing super type when optimizing %s", node); - Expression valueExpression = toExpression(value, valueType); - Expression patternExpression = toExpression(pattern, patternType); - Type superType = commonSuperType.get(); - if (!valueType.equals(superType)) { - valueExpression = new Cast(valueExpression, superType.getTypeSignature().toString(), false, typeManager.isTypeOnlyCoercion(valueType, superType)); - } - if (!patternType.equals(superType)) { - patternExpression = new Cast(patternExpression, superType.getTypeSignature().toString(), false, typeManager.isTypeOnlyCoercion(patternType, superType)); - } - return new ComparisonExpression(ComparisonExpressionType.EQUAL, valueExpression, patternExpression); + return new ComparisonExpression(ComparisonExpressionType.EQUAL, + toExpression(value, expressionTypes.get(node.getValue())), + toExpression(pattern, expressionTypes.get(node.getPattern()))); } } @@ -1270,13 +1225,11 @@ private SinglePagePositionContext(int position, Block[] blocks) this.blocks = blocks; } - @Override public Block getBlock(int channel) { return blocks[channel]; } - @Override public int getPosition(int channel) { return position; @@ -1382,22 +1335,4 @@ private static boolean isArray(Type type) { return type.getTypeSignature().getBase().equals(StandardTypes.ARRAY); } - - private static class LambdaSymbolResolver - implements SymbolResolver - { - private final Map values; - - public LambdaSymbolResolver(Map values) - { - this.values = requireNonNull(values, "values is null"); - } - - @Override - public Object getValue(Symbol symbol) - { - checkState(values.containsKey(symbol.getName()), "values does not contain %s", symbol); - return values.get(symbol.getName()); - } - } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/ExpressionSymbolInliner.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/ExpressionSymbolInliner.java index ea79a3ffd5fd..0f1fd21063d1 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/ExpressionSymbolInliner.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/ExpressionSymbolInliner.java @@ -16,7 +16,6 @@ import com.facebook.presto.sql.tree.Expression; import com.facebook.presto.sql.tree.ExpressionRewriter; import com.facebook.presto.sql.tree.ExpressionTreeRewriter; -import com.facebook.presto.sql.tree.LambdaExpression; import com.facebook.presto.sql.tree.SymbolReference; import java.util.Map; @@ -40,11 +39,4 @@ public Expression rewriteSymbolReference(SymbolReference node, Void context, Exp checkState(expression != null, "Cannot resolve symbol %s", node.getName()); return expression; } - - @Override - public Expression rewriteLambdaExpression(LambdaExpression node, Void context, ExpressionTreeRewriter treeRewriter) - { - // Lambda does not support capture yet. As a result, relation/columns can not exist in lambda. - return node; - } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/InterpretedInternalFilterFunction.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/InterpretedInternalFilterFunction.java index 221894a846e6..d31b69a9258d 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/InterpretedInternalFilterFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/InterpretedInternalFilterFunction.java @@ -22,6 +22,7 @@ import com.facebook.presto.spi.type.Type; import com.facebook.presto.sql.parser.SqlParser; import com.facebook.presto.sql.tree.Expression; +import com.facebook.presto.sql.tree.ExpressionTreeRewriter; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -48,7 +49,7 @@ public InterpretedInternalFilterFunction( Session session) { // pre-compute symbol -> input mappings and replace the corresponding nodes in the tree - Expression rewritten = new SymbolToInputRewriter(symbolToInputMappings).rewrite(predicate); + Expression rewritten = ExpressionTreeRewriter.rewriteWith(new SymbolToInputRewriter(symbolToInputMappings), predicate); // analyze expression so we can know the type of every expression in the tree ImmutableMap.Builder inputTypes = ImmutableMap.builder(); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/InterpretedProjectionFunction.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/InterpretedProjectionFunction.java index 85eb4625a5fe..9992da7e2065 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/InterpretedProjectionFunction.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/InterpretedProjectionFunction.java @@ -22,6 +22,7 @@ import com.facebook.presto.spi.type.Type; import com.facebook.presto.sql.parser.SqlParser; import com.facebook.presto.sql.tree.Expression; +import com.facebook.presto.sql.tree.ExpressionTreeRewriter; import com.google.common.collect.ImmutableMap; import io.airlift.slice.Slice; @@ -50,7 +51,7 @@ public InterpretedProjectionFunction( Session session) { // pre-compute symbol -> input mappings and replace the corresponding nodes in the tree - Expression rewritten = new SymbolToInputRewriter(symbolToInputMappings).rewrite(expression); + Expression rewritten = ExpressionTreeRewriter.rewriteWith(new SymbolToInputRewriter(symbolToInputMappings), expression); // analyze expression so we can know the type of every expression in the tree ImmutableMap.Builder inputTypes = ImmutableMap.builder(); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/LiteralInterpreter.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/LiteralInterpreter.java index 8f8b61eb2c2f..18b88e4f6800 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/LiteralInterpreter.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/LiteralInterpreter.java @@ -139,13 +139,13 @@ public static Expression toExpression(Object object, Type type) // if you remove this, you will need to update the TupleDomainOrcPredicate // When changing this, don't forget about similar code for REAL below if (value.isNaN()) { - return new FunctionCall(QualifiedName.of("nan"), ImmutableList.of()); + return new FunctionCall(QualifiedName.of("nan"), ImmutableList.of()); } else if (value.equals(Double.NEGATIVE_INFINITY)) { - return ArithmeticUnaryExpression.negative(new FunctionCall(QualifiedName.of("infinity"), ImmutableList.of())); + return ArithmeticUnaryExpression.negative(new FunctionCall(QualifiedName.of("infinity"), ImmutableList.of())); } else if (value.equals(Double.POSITIVE_INFINITY)) { - return new FunctionCall(QualifiedName.of("infinity"), ImmutableList.of()); + return new FunctionCall(QualifiedName.of("infinity"), ImmutableList.of()); } else { return new DoubleLiteral(object.toString()); @@ -283,7 +283,7 @@ protected Object visitGenericLiteral(GenericLiteral node, ConnectorSession sessi if (JSON.equals(type)) { ScalarFunctionImplementation operator = metadata.getFunctionRegistry().getScalarFunctionImplementation(new Signature("json_parse", SCALAR, JSON.getTypeSignature(), VARCHAR.getTypeSignature())); try { - return ExpressionInterpreter.invoke(session, operator, ImmutableList.of(utf8Slice(node.getValue()))); + return ExpressionInterpreter.invoke(session, operator, ImmutableList.of(utf8Slice(node.getValue()))); } catch (Throwable throwable) { throw Throwables.propagate(throwable); @@ -299,7 +299,7 @@ protected Object visitGenericLiteral(GenericLiteral node, ConnectorSession sessi throw new SemanticException(TYPE_MISMATCH, node, "No literal form for type %s", type); } try { - return ExpressionInterpreter.invoke(session, operator, ImmutableList.of(utf8Slice(node.getValue()))); + return ExpressionInterpreter.invoke(session, operator, ImmutableList.of(utf8Slice(node.getValue()))); } catch (Throwable throwable) { throw Throwables.propagate(throwable); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/LocalExecutionPlanner.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/LocalExecutionPlanner.java index eb6a0293709c..3037ec3a3e03 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/LocalExecutionPlanner.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/LocalExecutionPlanner.java @@ -139,6 +139,7 @@ import com.facebook.presto.sql.relational.SqlToRowExpressionTranslator; import com.facebook.presto.sql.tree.BooleanLiteral; import com.facebook.presto.sql.tree.Expression; +import com.facebook.presto.sql.tree.ExpressionTreeRewriter; import com.facebook.presto.sql.tree.FunctionCall; import com.facebook.presto.sql.tree.SymbolReference; import com.google.common.collect.HashMultimap; @@ -174,10 +175,8 @@ import java.util.function.Supplier; import java.util.stream.Collectors; -import static com.facebook.presto.SystemSessionProperties.getOperatorMemoryLimitBeforeSpill; import static com.facebook.presto.SystemSessionProperties.getTaskConcurrency; import static com.facebook.presto.SystemSessionProperties.getTaskWriterCount; -import static com.facebook.presto.SystemSessionProperties.isSpillEnabled; import static com.facebook.presto.metadata.FunctionKind.SCALAR; import static com.facebook.presto.operator.DistinctLimitOperator.DistinctLimitOperatorFactory; import static com.facebook.presto.operator.NestedLoopBuildOperator.NestedLoopBuildOperatorFactory; @@ -889,10 +888,7 @@ public PhysicalOperation visitAggregation(AggregationNode node, LocalExecutionPl return planGlobalAggregation(context.getNextOperatorId(), node, source); } - boolean spillEnabled = isSpillEnabled(context.getSession()); - DataSize memoryLimitBeforeSpill = getOperatorMemoryLimitBeforeSpill(context.getSession()); - - return planGroupByAggregation(node, source, context.getNextOperatorId(), spillEnabled, memoryLimitBeforeSpill); + return planGroupByAggregation(node, source, context.getNextOperatorId()); } @Override @@ -1003,11 +999,11 @@ private PhysicalOperation visitScanFilterAndProject( // compiler uses inputs instead of symbols, so rewrite the expressions first SymbolToInputRewriter symbolToInputRewriter = new SymbolToInputRewriter(sourceLayout); - Expression rewrittenFilter = symbolToInputRewriter.rewrite(filterExpression); + Expression rewrittenFilter = ExpressionTreeRewriter.rewriteWith(symbolToInputRewriter, filterExpression); List rewrittenProjections = new ArrayList<>(); for (Symbol symbol : outputSymbols) { - rewrittenProjections.add(symbolToInputRewriter.rewrite(projectionExpressions.get(symbol))); + rewrittenProjections.add(ExpressionTreeRewriter.rewriteWith(symbolToInputRewriter, projectionExpressions.get(symbol))); } IdentityHashMap expressionTypes = getExpressionTypesFromInput( @@ -1167,7 +1163,7 @@ public PhysicalOperation visitValues(ValuesNode node, LocalExecutionPlanContext context.getSession(), metadata, sqlParser, - ImmutableMap.of(), + ImmutableMap.of(), ImmutableList.copyOf(row), emptyList(), false); @@ -1550,7 +1546,8 @@ private JoinFilterFunctionFactory compileJoinFilterFunction( Map sourceTypes = joinSourcesLayout.entrySet().stream() .collect(toImmutableMap(Map.Entry::getValue, entry -> types.get(entry.getKey()))); - Expression rewrittenFilter = new SymbolToInputRewriter(joinSourcesLayout).rewrite(filterExpression); + SymbolToInputRewriter symbolToInputRewriter = new SymbolToInputRewriter(joinSourcesLayout); + Expression rewrittenFilter = ExpressionTreeRewriter.rewriteWith(symbolToInputRewriter, filterExpression); IdentityHashMap expressionTypes = getExpressionTypesFromInput( session, @@ -1852,12 +1849,7 @@ private PhysicalOperation planGlobalAggregation(int operatorId, AggregationNode return new PhysicalOperation(operatorFactory, outputMappings.build(), source); } - private PhysicalOperation planGroupByAggregation( - AggregationNode node, - PhysicalOperation source, - int operatorId, - boolean spillEnabled, - DataSize memoryLimitBeforeSpill) + private PhysicalOperation planGroupByAggregation(AggregationNode node, PhysicalOperation source, int operatorId) { List groupBySymbols = node.getGroupingKeys(); @@ -1919,10 +1911,7 @@ private PhysicalOperation planGroupByAggregation( hashChannel, node.getGroupIdSymbol().map(mappings::get), 10_000, - maxPartialAggregationMemorySize, - spillEnabled, - memoryLimitBeforeSpill, - spillerFactory); + maxPartialAggregationMemorySize); return new PhysicalOperation(operatorFactory, mappings, source); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/LogicalPlanner.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/LogicalPlanner.java index 82eed4f8a925..71b72c8117aa 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/LogicalPlanner.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/LogicalPlanner.java @@ -46,7 +46,6 @@ import com.facebook.presto.sql.tree.Explain; import com.facebook.presto.sql.tree.Expression; import com.facebook.presto.sql.tree.Insert; -import com.facebook.presto.sql.tree.LambdaArgumentDeclaration; import com.facebook.presto.sql.tree.NullLiteral; import com.facebook.presto.sql.tree.Query; import com.facebook.presto.sql.tree.Statement; @@ -55,7 +54,6 @@ import com.google.common.collect.ImmutableSet; import java.util.ArrayList; -import java.util.IdentityHashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -131,36 +129,30 @@ public Plan plan(Analysis analysis, Stage stage) } public PlanNode planStatement(Analysis analysis, Statement statement) - { - if (statement instanceof CreateTableAsSelect && analysis.isCreateTableAsSelectNoOp()) { - checkState(analysis.getCreateTableDestination().isPresent(), "Table destination is missing"); - List emptyRow = ImmutableList.of(); - PlanNode source = new ValuesNode(idAllocator.getNextId(), ImmutableList.of(), ImmutableList.of(emptyRow)); - return new OutputNode(idAllocator.getNextId(), source, ImmutableList.of(), ImmutableList.of()); - } - return createOutputPlan(planStatementWithoutOutput(analysis, statement), analysis); - } - - private RelationPlan planStatementWithoutOutput(Analysis analysis, Statement statement) { if (statement instanceof CreateTableAsSelect) { + checkState(analysis.getCreateTableDestination().isPresent(), "Table destination is missing"); if (analysis.isCreateTableAsSelectNoOp()) { - throw new PrestoException(NOT_SUPPORTED, "CREATE TABLE IF NOT EXISTS is not supported in this context " + statement.getClass().getSimpleName()); + List emptyRow = ImmutableList.of(); + PlanNode source = new ValuesNode(idAllocator.getNextId(), ImmutableList.of(), ImmutableList.of(emptyRow)); + return new OutputNode(idAllocator.getNextId(), source, ImmutableList.of(), ImmutableList.of()); + } + else { + return createOutputPlan(createTableCreationPlan(analysis, ((CreateTableAsSelect) statement).getQuery()), analysis); } - return createTableCreationPlan(analysis, ((CreateTableAsSelect) statement).getQuery()); } else if (statement instanceof Insert) { checkState(analysis.getInsert().isPresent(), "Insert handle is missing"); - return createInsertPlan(analysis, (Insert) statement); + return createOutputPlan(createInsertPlan(analysis, (Insert) statement), analysis); } else if (statement instanceof Delete) { - return createDeletePlan(analysis, (Delete) statement); + return createOutputPlan(createDeletePlan(analysis, (Delete) statement), analysis); } else if (statement instanceof Query) { - return createRelationPlan(analysis, (Query) statement); + return createOutputPlan(createRelationPlan(analysis, (Query) statement), analysis); } else if (statement instanceof Explain && ((Explain) statement).isAnalyze()) { - return createExplainAnalyzePlan(analysis, (Explain) statement); + return createOutputPlan(createExplainAnalyzePlan(analysis, (Explain) statement), analysis); } else { throw new PrestoException(NOT_SUPPORTED, "Unsupported statement type " + statement.getClass().getSimpleName()); @@ -169,9 +161,8 @@ else if (statement instanceof Explain && ((Explain) statement).isAnalyze()) { private RelationPlan createExplainAnalyzePlan(Analysis analysis, Explain statement) { - RelationPlan underlyingPlan = planStatementWithoutOutput(analysis, statement.getStatement()); - PlanNode root = underlyingPlan.getRoot(); Scope scope = analysis.getScope(statement); + PlanNode root = planStatement(analysis, statement.getStatement()); Symbol outputSymbol = symbolAllocator.newSymbol(scope.getRelationType().getFieldByIndex(0)); root = new ExplainAnalyzeNode(idAllocator.getNextId(), root, outputSymbol); return new RelationPlan(root, scope, ImmutableList.of(outputSymbol)); @@ -320,8 +311,8 @@ private RelationPlan createTableWriterPlan( private RelationPlan createDeletePlan(Analysis analysis, Delete node) { - DeleteNode deleteNode = new QueryPlanner(analysis, symbolAllocator, idAllocator, buildLambdaDeclarationToSymbolMap(analysis, symbolAllocator), metadata, session) - .plan(node); + QueryPlanner planner = new QueryPlanner(analysis, symbolAllocator, idAllocator, metadata, session); + DeleteNode deleteNode = planner.plan(node); List outputs = ImmutableList.of(symbolAllocator.newSymbol("rows", BIGINT)); TableFinishNode commitNode = new TableFinishNode(idAllocator.getNextId(), deleteNode, deleteNode.getTarget(), outputs); @@ -352,7 +343,7 @@ private PlanNode createOutputPlan(RelationPlan plan, Analysis analysis) private RelationPlan createRelationPlan(Analysis analysis, Query query) { - return new RelationPlanner(analysis, symbolAllocator, idAllocator, buildLambdaDeclarationToSymbolMap(analysis, symbolAllocator), metadata, session) + return new RelationPlanner(analysis, symbolAllocator, idAllocator, metadata, session) .process(query, null); } @@ -380,20 +371,4 @@ private static List getOutputTableColumns(RelationPlan plan) } return columns.build(); } - - private static IdentityHashMap buildLambdaDeclarationToSymbolMap(Analysis analysis, SymbolAllocator symbolAllocator) - { - IdentityHashMap resultMap = new IdentityHashMap<>(); - for (Map.Entry entry : analysis.getTypes().entrySet()) { - if (!(entry.getKey() instanceof LambdaArgumentDeclaration)) { - continue; - } - LambdaArgumentDeclaration lambdaArgumentDeclaration = (LambdaArgumentDeclaration) entry.getKey(); - if (resultMap.containsKey(lambdaArgumentDeclaration)) { - continue; - } - resultMap.put(lambdaArgumentDeclaration, symbolAllocator.newSymbol(lambdaArgumentDeclaration, entry.getValue())); - } - return resultMap; - } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanBuilder.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanBuilder.java index bda4b49034f5..56ba031a2e59 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanBuilder.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanBuilder.java @@ -44,7 +44,7 @@ public PlanBuilder(TranslationMap translations, PlanNode root, List public TranslationMap copyTranslations() { - TranslationMap translations = new TranslationMap(getRelationPlan(), getAnalysis(), getTranslations().getLambdaDeclarationToSymbolMap()); + TranslationMap translations = new TranslationMap(getRelationPlan(), getAnalysis()); translations.copyMappingsFrom(getTranslations()); return translations; } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanFragmenter.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanFragmenter.java index c0801ec7f28d..168641b1f552 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanFragmenter.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanFragmenter.java @@ -63,11 +63,7 @@ */ public class PlanFragmenter { - private PlanFragmenter() - { - } - - public static SubPlan createSubPlans(Session session, Metadata metadata, Plan plan) + public SubPlan createSubPlans(Session session, Metadata metadata, Plan plan) { Fragmenter fragmenter = new Fragmenter(session, metadata, plan.getSymbolAllocator().getTypes()); @@ -76,7 +72,6 @@ public static SubPlan createSubPlans(Session session, Metadata metadata, Plan pl PlanNode root = SimplePlanRewriter.rewriteWith(fragmenter, plan.getRoot(), properties); SubPlan result = fragmenter.buildRootFragment(root, properties); - checkState(result.getFragment().getPartitioning().isSingleNode(), "Root of PlanFragment is not single node"); result.sanityCheck(); return result; diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizers.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizers.java index 2d44d36863b0..38a9ee463323 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizers.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanOptimizers.java @@ -23,6 +23,7 @@ import com.facebook.presto.sql.planner.optimizations.CountConstantOptimizer; import com.facebook.presto.sql.planner.optimizations.DesugaringOptimizer; import com.facebook.presto.sql.planner.optimizations.EmptyDeleteOptimizer; +import com.facebook.presto.sql.planner.optimizations.EvaluateConstantApply; import com.facebook.presto.sql.planner.optimizations.HashGenerationOptimizer; import com.facebook.presto.sql.planner.optimizations.ImplementFilteredAggregations; import com.facebook.presto.sql.planner.optimizations.ImplementIntersectAndExceptAsUnion; @@ -42,13 +43,11 @@ import com.facebook.presto.sql.planner.optimizations.PruneIdentityProjections; import com.facebook.presto.sql.planner.optimizations.PruneUnreferencedOutputs; import com.facebook.presto.sql.planner.optimizations.PushTableWriteThroughUnion; -import com.facebook.presto.sql.planner.optimizations.RemoveUnreferencedScalarInputApplyNodes; +import com.facebook.presto.sql.planner.optimizations.RemoveRedundantApply; import com.facebook.presto.sql.planner.optimizations.SetFlatteningOptimizer; import com.facebook.presto.sql.planner.optimizations.SimplifyExpressions; import com.facebook.presto.sql.planner.optimizations.SingleDistinctOptimizer; import com.facebook.presto.sql.planner.optimizations.TransformCorrelatedScalarAggregationToJoin; -import com.facebook.presto.sql.planner.optimizations.TransformExistsApplyToScalarApply; -import com.facebook.presto.sql.planner.optimizations.TransformQuantifiedComparisonApplyToScalarApply; import com.facebook.presto.sql.planner.optimizations.TransformUncorrelatedInPredicateSubqueryToSemiJoin; import com.facebook.presto.sql.planner.optimizations.TransformUncorrelatedScalarToJoin; import com.facebook.presto.sql.planner.optimizations.UnaliasSymbolReferences; @@ -85,9 +84,6 @@ public PlanOptimizers(Metadata metadata, SqlParser sqlParser, FeaturesConfig fea new LimitPushDown(), // Run the LimitPushDown after flattening set operators to make it easier to do the set flattening new PruneUnreferencedOutputs(), new MergeProjections(), - new TransformExistsApplyToScalarApply(metadata), - new TransformQuantifiedComparisonApplyToScalarApply(metadata), - new RemoveUnreferencedScalarInputApplyNodes(), new TransformUncorrelatedInPredicateSubqueryToSemiJoin(), new TransformUncorrelatedScalarToJoin(), new TransformCorrelatedScalarAggregationToJoin(metadata), @@ -104,7 +100,9 @@ public PlanOptimizers(Metadata metadata, SqlParser sqlParser, FeaturesConfig fea new MergeProjections(), new PruneUnreferencedOutputs(), // Make sure to run this at the end to help clean the plan for logging/execution and not remove info that other optimizers might need at an earlier point new PruneIdentityProjections(), // This MUST run after PruneUnreferencedOutputs as it may introduce new redundant projections - new MetadataQueryOptimizer(metadata)); + new MetadataQueryOptimizer(metadata), + new EvaluateConstantApply(), + new RemoveRedundantApply()); if (featuresConfig.isOptimizeSingleDistinct()) { builder.add(new SingleDistinctOptimizer()); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanPrinter.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanPrinter.java index 9e4e17fdc685..f38c3974b728 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanPrinter.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/PlanPrinter.java @@ -76,11 +76,8 @@ import com.facebook.presto.sql.tree.ComparisonExpression; import com.facebook.presto.sql.tree.ComparisonExpressionType; import com.facebook.presto.sql.tree.Expression; -import com.facebook.presto.sql.tree.FrameBound; import com.facebook.presto.sql.tree.FunctionCall; import com.facebook.presto.sql.tree.SymbolReference; -import com.facebook.presto.sql.tree.Window; -import com.facebook.presto.sql.tree.WindowFrame; import com.facebook.presto.util.GraphvizPrinter; import com.google.common.base.CaseFormat; import com.google.common.base.Functions; @@ -428,7 +425,7 @@ public Void visitJoin(JoinNode node, Integer indent) node.getFilter().ifPresent(expression -> joinExpressions.add(expression)); // Check if the node is actually a cross join node - if (node.getType() == JoinNode.Type.INNER && joinExpressions.isEmpty()) { + if (node.getType() == JoinNode.Type.INNER && node.getCriteria().isEmpty()) { print(indent, "- CrossJoin => [%s]", formatOutputs(node.getOutputSymbols())); } else { @@ -626,12 +623,7 @@ public Void visitWindow(WindowNode node, Integer indent) for (Map.Entry entry : node.getWindowFunctions().entrySet()) { FunctionCall call = entry.getValue().getFunctionCall(); - String frameInfo = call.getWindow() - .flatMap(Window::getFrame) - .map(PlanPrinter::formatFrame) - .orElse(""); - - print(indent + 2, "%s := %s(%s) %s", entry.getKey(), call.getName(), Joiner.on(", ").join(call.getArguments()), frameInfo); + print(indent + 2, "%s := %s(%s)", entry.getKey(), call.getName(), Joiner.on(", ").join(call.getArguments())); } return processChildren(node, indent + 1); } @@ -688,7 +680,7 @@ public Void visitTableScan(TableScanNode node, Integer indent) TupleDomain predicate = node.getLayout() .map(layoutHandle -> metadata.getLayout(session, layoutHandle)) .map(TableLayout::getPredicate) - .orElse(TupleDomain.all()); + .orElse(TupleDomain.all()); if (node.getLayout().isPresent()) { // TODO: find a better way to do this @@ -751,7 +743,13 @@ public Void visitProject(ProjectNode node, Integer indent) { print(indent, "- Project => [%s]", formatOutputs(node.getOutputSymbols())); printStats(indent + 2, node.getId()); - printAssignments(node.getAssignments(), indent + 2); + for (Map.Entry entry : node.getAssignments().entrySet()) { + if (entry.getValue() instanceof SymbolReference && ((SymbolReference) entry.getValue()).getName().equals(entry.getKey().getName())) { + // skip identity assignments + continue; + } + print(indent + 2, "%s := %s", entry.getKey(), entry.getValue()); + } return processChildren(node, indent + 1); } @@ -934,7 +932,6 @@ public Void visitApply(ApplyNode node, Integer indent) { print(indent, "- Apply[%s] => [%s]", node.getCorrelation(), formatOutputs(node.getOutputSymbols())); printStats(indent + 2, node.getId()); - printAssignments(node.getSubqueryAssignments(), indent + 4); return processChildren(node, indent + 1); } @@ -954,17 +951,6 @@ private Void processChildren(PlanNode node, int indent) return null; } - private void printAssignments(Map assignments, int indent) - { - for (Map.Entry entry : assignments.entrySet()) { - if (entry.getValue() instanceof SymbolReference && ((SymbolReference) entry.getValue()).getName().equals(entry.getKey().getName())) { - // skip identity assignments - continue; - } - print(indent, "%s := %s", entry.getKey(), entry.getValue()); - } - } - private String formatOutputs(Iterable symbols) { return Joiner.on(", ").join(Iterables.transform(symbols, input -> input + ":" + types.get(input).getDisplayName())); @@ -1049,25 +1035,6 @@ private static String formatHash(Optional... hashes) return "[" + Joiner.on(", ").join(symbols) + "]"; } - private static String formatFrame(WindowFrame frame) - { - StringBuilder builder = new StringBuilder(frame.getType().toString()); - FrameBound start = frame.getStart(); - if (start.getValue().isPresent()) { - builder.append(" ").append(start.getOriginalValue().get()); - } - builder.append(" ").append(start.getType()); - - Optional end = frame.getEnd(); - if (end.isPresent()) { - if (end.get().getOriginalValue().isPresent()) { - builder.append(" ").append(end.get().getOriginalValue().get()); - } - builder.append(" ").append(end.get().getType()); - } - return builder.toString(); - } - private static String castToVarchar(Type type, Object value, Metadata metadata, Session session) { if (value == null) { diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/QueryPlanner.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/QueryPlanner.java index 100a4d4fef28..5a3b06fcd89b 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/QueryPlanner.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/QueryPlanner.java @@ -46,7 +46,6 @@ import com.facebook.presto.sql.tree.FieldReference; import com.facebook.presto.sql.tree.FrameBound; import com.facebook.presto.sql.tree.FunctionCall; -import com.facebook.presto.sql.tree.LambdaArgumentDeclaration; import com.facebook.presto.sql.tree.Node; import com.facebook.presto.sql.tree.Query; import com.facebook.presto.sql.tree.QuerySpecification; @@ -64,7 +63,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; -import java.util.IdentityHashMap; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; @@ -87,33 +85,24 @@ class QueryPlanner private final Analysis analysis; private final SymbolAllocator symbolAllocator; private final PlanNodeIdAllocator idAllocator; - private final IdentityHashMap lambdaDeclarationToSymbolMap; private final Metadata metadata; private final Session session; private final SubqueryPlanner subqueryPlanner; - QueryPlanner( - Analysis analysis, - SymbolAllocator symbolAllocator, - PlanNodeIdAllocator idAllocator, - IdentityHashMap lambdaDeclarationToSymbolMap, - Metadata metadata, - Session session) + QueryPlanner(Analysis analysis, SymbolAllocator symbolAllocator, PlanNodeIdAllocator idAllocator, Metadata metadata, Session session) { requireNonNull(analysis, "analysis is null"); requireNonNull(symbolAllocator, "symbolAllocator is null"); requireNonNull(idAllocator, "idAllocator is null"); - requireNonNull(lambdaDeclarationToSymbolMap, "lambdaDeclarationToSymbolMap is null"); requireNonNull(metadata, "metadata is null"); requireNonNull(session, "session is null"); this.analysis = analysis; this.symbolAllocator = symbolAllocator; this.idAllocator = idAllocator; - this.lambdaDeclarationToSymbolMap = lambdaDeclarationToSymbolMap; this.metadata = metadata; this.session = session; - this.subqueryPlanner = new SubqueryPlanner(analysis, symbolAllocator, idAllocator, lambdaDeclarationToSymbolMap, metadata, session, analysis.getParameters()); + this.subqueryPlanner = new SubqueryPlanner(analysis, symbolAllocator, idAllocator, metadata, session, analysis.getParameters()); } public RelationPlan plan(Query query) @@ -193,7 +182,7 @@ public DeleteNode plan(Delete node) Scope scope = Scope.builder().withRelationType(new RelationType(fields.build())).build(); RelationPlan relationPlan = new RelationPlan(tableScan, scope, outputSymbols.build()); - TranslationMap translations = new TranslationMap(relationPlan, analysis, lambdaDeclarationToSymbolMap); + TranslationMap translations = new TranslationMap(relationPlan, analysis); translations.setFieldMappings(relationPlan.getOutputSymbols()); PlanBuilder builder = new PlanBuilder(translations, relationPlan.getRoot(), analysis.getParameters()); @@ -211,7 +200,7 @@ public DeleteNode plan(Delete node) return new DeleteNode(idAllocator.getNextId(), builder.getRoot(), new DeleteHandle(handle, metadata.getTableMetadata(session, handle).getTable()), rowId, outputs); } - private static List computeOutputs(PlanBuilder builder, List outputExpressions) + private List computeOutputs(PlanBuilder builder, List outputExpressions) { ImmutableList.Builder outputSymbols = ImmutableList.builder(); for (Expression expression : outputExpressions) { @@ -222,10 +211,10 @@ private static List computeOutputs(PlanBuilder builder, List private PlanBuilder planQueryBody(Query query) { - RelationPlan relationPlan = new RelationPlanner(analysis, symbolAllocator, idAllocator, lambdaDeclarationToSymbolMap, metadata, session) + RelationPlan relationPlan = new RelationPlanner(analysis, symbolAllocator, idAllocator, metadata, session) .process(query.getQueryBody(), null); - TranslationMap translations = new TranslationMap(relationPlan, analysis, lambdaDeclarationToSymbolMap); + TranslationMap translations = new TranslationMap(relationPlan, analysis); // Make field->symbol mapping from underlying relation plan available for translations // This makes it possible to rewrite FieldOrExpressions that reference fields from the QuerySpecification directly @@ -239,14 +228,14 @@ private PlanBuilder planFrom(QuerySpecification node) RelationPlan relationPlan; if (node.getFrom().isPresent()) { - relationPlan = new RelationPlanner(analysis, symbolAllocator, idAllocator, lambdaDeclarationToSymbolMap, metadata, session) + relationPlan = new RelationPlanner(analysis, symbolAllocator, idAllocator, metadata, session) .process(node.getFrom().get(), null); } else { relationPlan = planImplicitTable(node); } - TranslationMap translations = new TranslationMap(relationPlan, analysis, lambdaDeclarationToSymbolMap); + TranslationMap translations = new TranslationMap(relationPlan, analysis); // Make field->symbol mapping from underlying relation plan available for translations // This makes it possible to rewrite FieldOrExpressions that reference fields from the FROM clause directly @@ -260,9 +249,9 @@ private RelationPlan planImplicitTable(QuerySpecification node) List emptyRow = ImmutableList.of(); Scope scope = Scope.builder().withParent(analysis.getScope(node)).build(); return new RelationPlan( - new ValuesNode(idAllocator.getNextId(), ImmutableList.of(), ImmutableList.of(emptyRow)), + new ValuesNode(idAllocator.getNextId(), ImmutableList.of(), ImmutableList.of(emptyRow)), scope, - ImmutableList.of()); + ImmutableList.of()); } private PlanBuilder filter(PlanBuilder subPlan, Expression predicate, Node node) @@ -283,7 +272,7 @@ private PlanBuilder filter(PlanBuilder subPlan, Expression predicate, Node node) private PlanBuilder project(PlanBuilder subPlan, Iterable expressions) { - TranslationMap outputTranslations = new TranslationMap(subPlan.getRelationPlan(), analysis, lambdaDeclarationToSymbolMap); + TranslationMap outputTranslations = new TranslationMap(subPlan.getRelationPlan(), analysis); ImmutableMap.Builder projections = ImmutableMap.builder(); for (Expression expression : expressions) { @@ -328,7 +317,7 @@ private Map coerce(Iterable expression private PlanBuilder explicitCoercionFields(PlanBuilder subPlan, Iterable alreadyCoerced, Iterable uncoerced) { - TranslationMap translations = new TranslationMap(subPlan.getRelationPlan(), analysis, lambdaDeclarationToSymbolMap); + TranslationMap translations = new TranslationMap(subPlan.getRelationPlan(), analysis); ImmutableMap.Builder projections = ImmutableMap.builder(); projections.putAll(coerce(uncoerced, subPlan, translations)); @@ -402,7 +391,7 @@ private PlanBuilder aggregate(PlanBuilder subPlan, QuerySpecification node) // 2. Aggregate // 2.a. Rewrite aggregate arguments - TranslationMap argumentTranslations = new TranslationMap(subPlan.getRelationPlan(), analysis, lambdaDeclarationToSymbolMap); + TranslationMap argumentTranslations = new TranslationMap(subPlan.getRelationPlan(), analysis); ImmutableMap.Builder argumentMappingBuilder = ImmutableMap.builder(); for (Expression argument : arguments.build()) { Expression parametersReplaced = ExpressionTreeRewriter.rewriteWith(new ParameterRewriter(analysis.getParameters(), analysis), argument); @@ -418,7 +407,7 @@ private PlanBuilder aggregate(PlanBuilder subPlan, QuerySpecification node) Map argumentMappings = argumentMappingBuilder.build(); // 2.b. Rewrite grouping columns - TranslationMap groupingTranslations = new TranslationMap(subPlan.getRelationPlan(), analysis, lambdaDeclarationToSymbolMap); + TranslationMap groupingTranslations = new TranslationMap(subPlan.getRelationPlan(), analysis); Map groupingSetMappings = new HashMap<>(); List> groupingSymbols = new ArrayList<>(); @@ -465,7 +454,7 @@ private PlanBuilder aggregate(PlanBuilder subPlan, QuerySpecification node) subPlan = new PlanBuilder(groupingTranslations, project, analysis.getParameters()); } - TranslationMap aggregationTranslations = new TranslationMap(subPlan.getRelationPlan(), analysis, lambdaDeclarationToSymbolMap); + TranslationMap aggregationTranslations = new TranslationMap(subPlan.getRelationPlan(), analysis); aggregationTranslations.copyMappingsFrom(groupingTranslations); // 2.d. Rewrite aggregates @@ -776,7 +765,7 @@ private PlanBuilder limit(PlanBuilder subPlan, List orderBy, Optional< return subPlan; } - private static SortOrder toSortOrder(SortItem sortItem) + private SortOrder toSortOrder(SortItem sortItem) { if (sortItem.getOrdering() == Ordering.ASCENDING) { if (sortItem.getNullOrdering() == NullOrdering.FIRST) { diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/RelationPlanner.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/RelationPlanner.java index da8b07d0c17a..a0cbd97f9ba9 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/RelationPlanner.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/RelationPlanner.java @@ -15,6 +15,7 @@ import com.facebook.presto.Session; import com.facebook.presto.metadata.Metadata; +import com.facebook.presto.metadata.Signature; import com.facebook.presto.metadata.TableHandle; import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.predicate.TupleDomain; @@ -45,11 +46,10 @@ import com.facebook.presto.sql.tree.Except; import com.facebook.presto.sql.tree.Expression; import com.facebook.presto.sql.tree.ExpressionTreeRewriter; +import com.facebook.presto.sql.tree.FunctionCall; import com.facebook.presto.sql.tree.InPredicate; import com.facebook.presto.sql.tree.Intersect; import com.facebook.presto.sql.tree.Join; -import com.facebook.presto.sql.tree.JoinUsing; -import com.facebook.presto.sql.tree.LambdaArgumentDeclaration; import com.facebook.presto.sql.tree.LongLiteral; import com.facebook.presto.sql.tree.QualifiedName; import com.facebook.presto.sql.tree.Query; @@ -74,7 +74,6 @@ import com.google.common.collect.UnmodifiableIterator; import java.util.ArrayList; -import java.util.IdentityHashMap; import java.util.Iterator; import java.util.List; import java.util.Optional; @@ -96,33 +95,24 @@ class RelationPlanner private final Analysis analysis; private final SymbolAllocator symbolAllocator; private final PlanNodeIdAllocator idAllocator; - private final IdentityHashMap lambdaDeclarationToSymbolMap; private final Metadata metadata; private final Session session; private final SubqueryPlanner subqueryPlanner; - RelationPlanner( - Analysis analysis, - SymbolAllocator symbolAllocator, - PlanNodeIdAllocator idAllocator, - IdentityHashMap lambdaDeclarationToSymbolMap, - Metadata metadata, - Session session) + RelationPlanner(Analysis analysis, SymbolAllocator symbolAllocator, PlanNodeIdAllocator idAllocator, Metadata metadata, Session session) { requireNonNull(analysis, "analysis is null"); requireNonNull(symbolAllocator, "symbolAllocator is null"); requireNonNull(idAllocator, "idAllocator is null"); - requireNonNull(lambdaDeclarationToSymbolMap, "lambdaDeclarationToSymbolMap is null"); requireNonNull(metadata, "metadata is null"); requireNonNull(session, "session is null"); this.analysis = analysis; this.symbolAllocator = symbolAllocator; this.idAllocator = idAllocator; - this.lambdaDeclarationToSymbolMap = lambdaDeclarationToSymbolMap; this.metadata = metadata; this.session = session; - this.subqueryPlanner = new SubqueryPlanner(analysis, symbolAllocator, idAllocator, lambdaDeclarationToSymbolMap, metadata, session, analysis.getParameters()); + this.subqueryPlanner = new SubqueryPlanner(analysis, symbolAllocator, idAllocator, metadata, session, analysis.getParameters()); } @Override @@ -234,29 +224,19 @@ protected RelationPlan visitJoin(Join node, Void context) continue; } - Set dependencies = DependencyExtractor.extractNames(conjunct, analysis.getColumnReferences()); - boolean isJoinUsing = node.getCriteria().filter(JoinUsing.class::isInstance).isPresent(); - if (!isJoinUsing && (dependencies.stream().allMatch(left::canResolve) || dependencies.stream().allMatch(right::canResolve))) { - // If the conjunct can be evaluated entirely with the inputs on either side of the join, add - // it to the list complex expressions and let the optimizers figure out how to push it down later. - // Due to legacy reasons, the expression for "join using" looks like "x = x", which (incorrectly) - // appears to fit the condition we're after. So we skip them. - - complexJoinExpressions.add(conjunct); - } - else if (conjunct instanceof ComparisonExpression) { + if (conjunct instanceof ComparisonExpression) { Expression firstExpression = ((ComparisonExpression) conjunct).getLeft(); Expression secondExpression = ((ComparisonExpression) conjunct).getRight(); ComparisonExpressionType comparisonType = ((ComparisonExpression) conjunct).getType(); Set firstDependencies = DependencyExtractor.extractNames(firstExpression, analysis.getColumnReferences()); Set secondDependencies = DependencyExtractor.extractNames(secondExpression, analysis.getColumnReferences()); - if (firstDependencies.stream().allMatch(left::canResolve) && secondDependencies.stream().allMatch(right::canResolve)) { + if (firstDependencies.stream().allMatch(left.canResolvePredicate()) && secondDependencies.stream().allMatch(right.canResolvePredicate())) { leftComparisonExpressions.add(firstExpression); rightComparisonExpressions.add(secondExpression); joinConditionComparisonTypes.add(comparisonType); } - else if (firstDependencies.stream().allMatch(right::canResolve) && secondDependencies.stream().allMatch(left::canResolve)) { + else if (firstDependencies.stream().allMatch(right.canResolvePredicate()) && secondDependencies.stream().allMatch(left.canResolvePredicate())) { leftComparisonExpressions.add(secondExpression); rightComparisonExpressions.add(firstExpression); joinConditionComparisonTypes.add(comparisonType.flip()); @@ -316,7 +296,7 @@ else if (firstDependencies.stream().allMatch(right::canResolve) && secondDepende } RelationPlan intermediateRootRelationPlan = new RelationPlan(root, analysis.getScope(node), outputSymbols); - TranslationMap translationMap = new TranslationMap(intermediateRootRelationPlan, analysis, lambdaDeclarationToSymbolMap); + TranslationMap translationMap = new TranslationMap(intermediateRootRelationPlan, analysis); translationMap.setFieldMappings(outputSymbols); translationMap.putExpressionMappingsFrom(leftPlanBuilder.getTranslations()); translationMap.putExpressionMappingsFrom(rightPlanBuilder.getTranslations()); @@ -341,7 +321,6 @@ else if (firstDependencies.stream().allMatch(right::canResolve) && secondDepende rootPlanBuilder = subqueryPlanner.handleSubqueries(rootPlanBuilder, complexJoinExpressions, node); for (Expression expression : complexJoinExpressions) { - expression = ExpressionTreeRewriter.rewriteWith(new ParameterRewriter(analysis.getParameters(), analysis), expression); postInnerJoinConditions.add(rootPlanBuilder.rewrite(expression)); } root = rootPlanBuilder.getRoot(); @@ -356,7 +335,7 @@ else if (firstDependencies.stream().allMatch(right::canResolve) && secondDepende return new RelationPlan(root, analysis.getScope(node), outputSymbols); } - private static boolean isEqualComparisonExpression(Expression conjunct) + private boolean isEqualComparisonExpression(Expression conjunct) { return conjunct instanceof ComparisonExpression && ((ComparisonExpression) conjunct).getType() == ComparisonExpressionType.EQUAL; } @@ -419,14 +398,14 @@ protected RelationPlan visitTableSubquery(TableSubquery node, Void context) @Override protected RelationPlan visitQuery(Query node, Void context) { - return new QueryPlanner(analysis, symbolAllocator, idAllocator, lambdaDeclarationToSymbolMap, metadata, session) + return new QueryPlanner(analysis, symbolAllocator, idAllocator, metadata, session) .plan(node); } @Override protected RelationPlan visitQuerySpecification(QuerySpecification node, Void context) { - return new QueryPlanner(analysis, symbolAllocator, idAllocator, lambdaDeclarationToSymbolMap, metadata, session) + return new QueryPlanner(analysis, symbolAllocator, idAllocator, metadata, session) .plan(node); } @@ -500,9 +479,9 @@ else if (type instanceof MapType) { } Optional ordinalitySymbol = node.isWithOrdinality() ? Optional.of(unnestedSymbolsIterator.next()) : Optional.empty(); checkState(!unnestedSymbolsIterator.hasNext(), "Not all output symbols were matched with input symbols"); - ValuesNode valuesNode = new ValuesNode(idAllocator.getNextId(), argumentSymbols.build(), ImmutableList.of(values.build())); + ValuesNode valuesNode = new ValuesNode(idAllocator.getNextId(), argumentSymbols.build(), ImmutableList.>of(values.build())); - UnnestNode unnestNode = new UnnestNode(idAllocator.getNextId(), valuesNode, ImmutableList.of(), unnestSymbols.build(), ordinalitySymbol); + UnnestNode unnestNode = new UnnestNode(idAllocator.getNextId(), valuesNode, ImmutableList.of(), unnestSymbols.build(), ordinalitySymbol); return new RelationPlan(unnestNode, scope, unnestedSymbols); } @@ -637,7 +616,7 @@ private SetOperationPlan process(SetOperation node) private PlanBuilder initializePlanBuilder(RelationPlan relationPlan) { - TranslationMap translations = new TranslationMap(relationPlan, analysis, lambdaDeclarationToSymbolMap); + TranslationMap translations = new TranslationMap(relationPlan, analysis); // Make field->symbol mapping from underlying relation plan available for translations // This makes it possible to rewrite FieldOrExpressions that reference fields from the underlying tuple directly @@ -650,9 +629,9 @@ private PlanNode distinct(PlanNode node) { return new AggregationNode(idAllocator.getNextId(), node, - ImmutableMap.of(), - ImmutableMap.of(), - ImmutableMap.of(), + ImmutableMap.of(), + ImmutableMap.of(), + ImmutableMap.of(), ImmutableList.of(node.getOutputSymbols()), AggregationNode.Step.SINGLE, Optional.empty(), diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/StageExecutionPlan.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/StageExecutionPlan.java index 8aa27ef189ff..e247b205380e 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/StageExecutionPlan.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/StageExecutionPlan.java @@ -43,7 +43,7 @@ public StageExecutionPlan( this.subStages = ImmutableList.copyOf(requireNonNull(subStages, "dependencies is null")); fieldNames = (fragment.getRoot() instanceof OutputNode) ? - Optional.of(ImmutableList.copyOf(((OutputNode) fragment.getRoot()).getColumnNames())) : + Optional.>of(ImmutableList.copyOf(((OutputNode) fragment.getRoot()).getColumnNames())) : Optional.empty(); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/SubqueryPlanner.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/SubqueryPlanner.java index 0334319ce9f8..5be088bacd3c 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/SubqueryPlanner.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/SubqueryPlanner.java @@ -14,27 +14,36 @@ package com.facebook.presto.sql.planner; import com.facebook.presto.Session; +import com.facebook.presto.metadata.FunctionRegistry; import com.facebook.presto.metadata.Metadata; +import com.facebook.presto.spi.type.Type; +import com.facebook.presto.spi.type.TypeSignature; import com.facebook.presto.sql.analyzer.Analysis; import com.facebook.presto.sql.planner.optimizations.Predicates; import com.facebook.presto.sql.planner.plan.AggregationNode; import com.facebook.presto.sql.planner.plan.ApplyNode; import com.facebook.presto.sql.planner.plan.EnforceSingleRowNode; import com.facebook.presto.sql.planner.plan.FilterNode; +import com.facebook.presto.sql.planner.plan.LimitNode; import com.facebook.presto.sql.planner.plan.PlanNode; import com.facebook.presto.sql.planner.plan.ProjectNode; import com.facebook.presto.sql.planner.plan.SimplePlanRewriter; import com.facebook.presto.sql.planner.plan.ValuesNode; import com.facebook.presto.sql.tree.BooleanLiteral; +import com.facebook.presto.sql.tree.Cast; +import com.facebook.presto.sql.tree.ComparisonExpression; import com.facebook.presto.sql.tree.DefaultExpressionTraversalVisitor; import com.facebook.presto.sql.tree.DereferenceExpression; import com.facebook.presto.sql.tree.ExistsPredicate; import com.facebook.presto.sql.tree.Expression; import com.facebook.presto.sql.tree.ExpressionTreeRewriter; +import com.facebook.presto.sql.tree.FunctionCall; import com.facebook.presto.sql.tree.InPredicate; -import com.facebook.presto.sql.tree.LambdaArgumentDeclaration; +import com.facebook.presto.sql.tree.LogicalBinaryExpression; +import com.facebook.presto.sql.tree.LongLiteral; import com.facebook.presto.sql.tree.Node; import com.facebook.presto.sql.tree.NotExpression; +import com.facebook.presto.sql.tree.QualifiedName; import com.facebook.presto.sql.tree.QualifiedNameReference; import com.facebook.presto.sql.tree.QuantifiedComparisonExpression; import com.facebook.presto.sql.tree.QuantifiedComparisonExpression.Quantifier; @@ -46,21 +55,27 @@ import com.google.common.collect.ImmutableSet; import java.util.Collection; -import java.util.IdentityHashMap; +import java.util.EnumSet; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; +import static com.facebook.presto.spi.type.BigintType.BIGINT; import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; import static com.facebook.presto.sql.analyzer.SemanticExceptions.throwNotSupportedException; import static com.facebook.presto.sql.planner.ExpressionNodeInliner.replaceExpression; import static com.facebook.presto.sql.planner.optimizations.PlanNodeSearcher.searchFrom; import static com.facebook.presto.sql.tree.ComparisonExpressionType.EQUAL; +import static com.facebook.presto.sql.tree.ComparisonExpressionType.GREATER_THAN; +import static com.facebook.presto.sql.tree.ComparisonExpressionType.GREATER_THAN_OR_EQUAL; +import static com.facebook.presto.sql.tree.ComparisonExpressionType.LESS_THAN; +import static com.facebook.presto.sql.tree.ComparisonExpressionType.LESS_THAN_OR_EQUAL; import static com.facebook.presto.sql.util.AstUtils.nodeContains; import static com.facebook.presto.util.ImmutableCollectors.toImmutableList; import static com.facebook.presto.util.ImmutableCollectors.toImmutableMap; import static com.facebook.presto.util.ImmutableCollectors.toImmutableSet; +import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import static com.google.common.collect.Iterables.getOnlyElement; import static java.lang.String.format; @@ -71,24 +86,15 @@ class SubqueryPlanner private final Analysis analysis; private final SymbolAllocator symbolAllocator; private final PlanNodeIdAllocator idAllocator; - private final IdentityHashMap lambdaDeclarationToSymbolMap; private final Metadata metadata; private final Session session; private final List parameters; - SubqueryPlanner( - Analysis analysis, - SymbolAllocator symbolAllocator, - PlanNodeIdAllocator idAllocator, - IdentityHashMap lambdaDeclarationToSymbolMap, - Metadata metadata, - Session session, - List parameters) + SubqueryPlanner(Analysis analysis, SymbolAllocator symbolAllocator, PlanNodeIdAllocator idAllocator, Metadata metadata, Session session, List parameters) { requireNonNull(analysis, "analysis is null"); requireNonNull(symbolAllocator, "symbolAllocator is null"); requireNonNull(idAllocator, "idAllocator is null"); - requireNonNull(lambdaDeclarationToSymbolMap, "lambdaDeclarationToSymbolMap is null"); requireNonNull(metadata, "metadata is null"); requireNonNull(session, "session is null"); requireNonNull(parameters, "parameters is null"); @@ -96,7 +102,6 @@ class SubqueryPlanner this.analysis = analysis; this.symbolAllocator = symbolAllocator; this.idAllocator = idAllocator; - this.lambdaDeclarationToSymbolMap = lambdaDeclarationToSymbolMap; this.metadata = metadata; this.session = session; this.parameters = parameters; @@ -178,16 +183,25 @@ private PlanBuilder appendInPredicateApplyNode(PlanBuilder subPlan, InPredicate checkState(inPredicate.getValueList() instanceof SubqueryExpression); PlanNode subquery = createRelationPlan(((SubqueryExpression) inPredicate.getValueList()).getQuery()).getRoot(); + Map correlation = extractCorrelation(subPlan, subquery); + if (!correlationAllowed && correlation.isEmpty()) { + throwNotSupportedException(inPredicate, "Correlated subquery in given context"); + } + subPlan = subPlan.appendProjections(correlation.keySet(), symbolAllocator, idAllocator); + subquery = replaceExpressionsWithSymbols(subquery, correlation); + TranslationMap translationMap = subPlan.copyTranslations(); InPredicate parametersReplaced = ExpressionTreeRewriter.rewriteWith(new ParameterRewriter(parameters, analysis), inPredicate); + translationMap.addIntermediateMapping(inPredicate, parametersReplaced); SymbolReference valueList = getOnlyElement(subquery.getOutputSymbols()).toSymbolReference(); + translationMap.addIntermediateMapping(parametersReplaced, new InPredicate(parametersReplaced.getValue(), valueList)); - InPredicate inPredicateSubqueryExpression = new InPredicate(subPlan.translate(parametersReplaced.getValue()).toSymbolReference(), valueList); - Symbol inPredicateSubquerySymbol = symbolAllocator.newSymbol(inPredicateSubqueryExpression, BOOLEAN); - subPlan.getTranslations().put(parametersReplaced, inPredicateSubquerySymbol); - subPlan.getTranslations().put(inPredicate, inPredicateSubquerySymbol); - - return appendApplyNode(subPlan, inPredicate, subquery, ImmutableMap.of(inPredicateSubquerySymbol, inPredicateSubqueryExpression), correlationAllowed); + return new PlanBuilder(translationMap, + new ApplyNode(idAllocator.getNextId(), + subPlan.getRoot(), + subquery, + ImmutableList.copyOf(correlation.values())), + analysis.getParameters()); } private PlanBuilder appendScalarSubqueryApplyNodes(PlanBuilder builder, Set scalarSubqueries, boolean correlationAllowed) @@ -205,13 +219,11 @@ private PlanBuilder appendScalarSubqueryApplyNode(PlanBuilder subPlan, SubqueryE return subPlan; } - PlanNode subqueryNode = new EnforceSingleRowNode(idAllocator.getNextId(), createRelationPlan(scalarSubquery.getQuery()).getRoot()); - subPlan.getTranslations().put(scalarSubquery, getOnlyElement(subqueryNode.getOutputSymbols())); - return appendApplyNode( + return appendSubqueryApplyNode( subPlan, + scalarSubquery, scalarSubquery.getQuery(), - subqueryNode, - identityAssigments(subqueryNode), + new EnforceSingleRowNode(idAllocator.getNextId(), createRelationPlan(scalarSubquery.getQuery()).getRoot()), correlationAllowed); } @@ -226,10 +238,11 @@ private PlanBuilder appendExistsSubqueryApplyNodes(PlanBuilder builder, Set - * - Project($0 > 0) - * - Aggregation(COUNT(*)) - * - Limit(1) - * -- subquery + * - EnforceSingleRow + * - Project($0 > 0) + * - Aggregation(COUNT(*)) + * - Limit(1) + * -- subquery * */ private PlanBuilder appendExistSubqueryApplyNode(PlanBuilder subPlan, ExistsPredicate existsPredicate, boolean correlationAllowed) @@ -241,19 +254,37 @@ private PlanBuilder appendExistSubqueryApplyNode(PlanBuilder subPlan, ExistsPred PlanNode subqueryPlan = createRelationPlan(existsPredicate.getSubquery()).getRoot(); + Symbol exists = symbolAllocator.newSymbol("exists", BOOLEAN); if (isAggregationWithEmptyGroupBy(subqueryPlan)) { subPlan.getTranslations().put(existsPredicate, BooleanLiteral.TRUE_LITERAL); return subPlan; } - Symbol exists = symbolAllocator.newSymbol("exists", BOOLEAN); - subPlan.getTranslations().put(existsPredicate, exists); - return appendApplyNode( - subPlan, - existsPredicate.getSubquery(), + subqueryPlan = new LimitNode(idAllocator.getNextId(), subqueryPlan, 1, false); + + FunctionRegistry functionRegistry = metadata.getFunctionRegistry(); + QualifiedName countFunction = QualifiedName.of("count"); + Symbol count = symbolAllocator.newSymbol(countFunction.toString(), BIGINT); + subqueryPlan = new AggregationNode( + idAllocator.getNextId(), subqueryPlan, - ImmutableMap.of(exists, existsPredicate), - correlationAllowed); + ImmutableMap.of(count, new FunctionCall(countFunction, ImmutableList.of())), + ImmutableMap.of(count, functionRegistry.resolveFunction(countFunction, ImmutableList.of())), + ImmutableMap.of(), + ImmutableList.of(ImmutableList.of()), + AggregationNode.Step.SINGLE, + Optional.empty(), + Optional.empty()); + + ComparisonExpression countGreaterThanZero = new ComparisonExpression(GREATER_THAN, count.toSymbolReference(), new Cast(new LongLiteral("0"), BIGINT.toString())); + subqueryPlan = new EnforceSingleRowNode( + idAllocator.getNextId(), + new ProjectNode( + idAllocator.getNextId(), + subqueryPlan, + ImmutableMap.of(exists, countGreaterThanZero))); + + return appendSubqueryApplyNode(subPlan, existsPredicate, existsPredicate.getSubquery(), subqueryPlan, correlationAllowed); } private PlanBuilder appendQuantifiedComparisonApplyNodes(PlanBuilder subPlan, Set quantifiedComparisons, boolean correlationAllowed) @@ -274,14 +305,13 @@ private PlanBuilder appendQuantifiedComparisonApplyNode(PlanBuilder subPlan, Qua case EQUAL: switch (quantifiedComparison.getQuantifier()) { case ALL: - return planQuantifiedApplyNode(subPlan, quantifiedComparison, correlationAllowed); + return planQuantifiedEqualsAll(subPlan, quantifiedComparison, correlationAllowed); case ANY: case SOME: // A = ANY B <=> A IN B InPredicate inPredicate = new InPredicate(quantifiedComparison.getValue(), quantifiedComparison.getSubquery()); - subPlan = appendInPredicateApplyNode(subPlan, inPredicate, correlationAllowed); - subPlan.getTranslations().put(quantifiedComparison, subPlan.translate(inPredicate)); - return subPlan; + subPlan.getTranslations().addIntermediateMapping(quantifiedComparison, inPredicate); + return appendInPredicateApplyNode(subPlan, inPredicate, correlationAllowed); } break; @@ -319,35 +349,115 @@ private PlanBuilder appendQuantifiedComparisonApplyNode(PlanBuilder subPlan, Qua case LESS_THAN_OR_EQUAL: case GREATER_THAN: case GREATER_THAN_OR_EQUAL: - return planQuantifiedApplyNode(subPlan, quantifiedComparison, correlationAllowed); + return planQuantifiedOrderable(subPlan, quantifiedComparison, correlationAllowed); } // all cases are checked, so this exception should never be thrown throw new IllegalArgumentException( format("Unexpected quantified comparison: '%s %s'", quantifiedComparison.getComparisonType().getValue(), quantifiedComparison.getQuantifier())); } - private PlanBuilder planQuantifiedApplyNode(PlanBuilder subPlan, QuantifiedComparisonExpression quantifiedComparison, boolean correlationAllowed) + private PlanBuilder planQuantifiedEqualsAll(PlanBuilder subPlan, QuantifiedComparisonExpression quantifiedComparison, boolean correlationAllowed) { + checkArgument(quantifiedComparison.getComparisonType() == EQUAL && quantifiedComparison.getQuantifier() == Quantifier.ALL); + // A = ALL B <=> min B = max B && A = min B RelationPlan subqueryRelationPlan = createRelationPlan(quantifiedComparison.getSubquery()); PlanNode subqueryPlan = subqueryRelationPlan.getRoot(); - SymbolReference subquerySymbolReference = getOnlyElement(subqueryRelationPlan.getOutputSymbols()).toSymbolReference(); - subPlan = subPlan.appendProjections(ImmutableList.of(quantifiedComparison.getValue()), symbolAllocator, idAllocator); - QuantifiedComparisonExpression subqueryQuantifiedComparison = new QuantifiedComparisonExpression( - quantifiedComparison.getComparisonType(), - quantifiedComparison.getQuantifier(), - subPlan.translate(quantifiedComparison.getValue()).toSymbolReference(), - subquerySymbolReference); - Symbol subqueryQuantifiedComparisonSymbol = symbolAllocator.newSymbol(subqueryQuantifiedComparison, BOOLEAN); - subPlan.getTranslations().put(quantifiedComparison, subqueryQuantifiedComparisonSymbol); - return appendApplyNode( - subPlan, - quantifiedComparison.getSubquery(), + List outputColumnReference = ImmutableList.of(getOnlyElement(subqueryRelationPlan.getOutputSymbols()).toSymbolReference()); + Type outputColumnType = getOnlyElement(subqueryRelationPlan.getDescriptor().getAllFields()).getType(); + if (!outputColumnType.isOrderable()) { + throwNotSupportedException(quantifiedComparison, "Quantified comparison '= ALL' or '<> ANY' for unorderable type " + outputColumnType.getDisplayName()); + } + List outputColumnTypeSignature = ImmutableList.of(outputColumnType.getTypeSignature()); + FunctionRegistry functionRegistry = metadata.getFunctionRegistry(); + QualifiedName min = QualifiedName.of("min"); + QualifiedName max = QualifiedName.of("max"); + Symbol minValue = symbolAllocator.newSymbol(min.toString(), outputColumnType); + Symbol maxValue = symbolAllocator.newSymbol(max.toString(), outputColumnType); + subqueryPlan = new AggregationNode( + idAllocator.getNextId(), subqueryPlan, - ImmutableMap.of(subqueryQuantifiedComparisonSymbol, subqueryQuantifiedComparison), - correlationAllowed); + ImmutableMap.of( + minValue, new FunctionCall(min, outputColumnReference), + maxValue, new FunctionCall(max, outputColumnReference) + ), + ImmutableMap.of( + minValue, functionRegistry.resolveFunction(min, outputColumnTypeSignature), + maxValue, functionRegistry.resolveFunction(max, outputColumnTypeSignature) + ), + ImmutableMap.of(), + ImmutableList.of(ImmutableList.of()), + AggregationNode.Step.SINGLE, + Optional.empty(), + Optional.empty()); + subqueryPlan = new EnforceSingleRowNode(idAllocator.getNextId(), subqueryPlan); + LogicalBinaryExpression valueComparedToSubquery = new LogicalBinaryExpression(LogicalBinaryExpression.Type.AND, + new ComparisonExpression(EQUAL, minValue.toSymbolReference(), maxValue.toSymbolReference()), + new ComparisonExpression(EQUAL, quantifiedComparison.getValue(), minValue.toSymbolReference()) + ); + subPlan.getTranslations().addIntermediateMapping(quantifiedComparison, valueComparedToSubquery); + return appendSubqueryApplyNode(subPlan, quantifiedComparison.getSubquery(), quantifiedComparison.getSubquery(), subqueryPlan, correlationAllowed); + } + + private PlanBuilder planQuantifiedOrderable(PlanBuilder subPlan, QuantifiedComparisonExpression quantifiedComparison, boolean correlationAllowed) + { + checkArgument(EnumSet.of(LESS_THAN, LESS_THAN_OR_EQUAL, GREATER_THAN, GREATER_THAN_OR_EQUAL).contains(quantifiedComparison.getComparisonType())); + // A < ALL B <=> A < min B + // A > ALL B <=> A > max B + // A < ANY B <=> A < max B + // A > ANY B <=> A > min B + QualifiedName aggregationFunction = chooseAggregationFunction(quantifiedComparison); + RelationPlan subqueryRelationPlan = createRelationPlan(quantifiedComparison.getSubquery()); + PlanNode subqueryPlan = subqueryRelationPlan.getRoot(); + Symbol outputColumn = getOnlyElement(subqueryRelationPlan.getOutputSymbols()); + Type outputColumnType = getOnlyElement(subqueryRelationPlan.getDescriptor().getAllFields()).getType(); + checkState(outputColumnType.isOrderable(), "Subquery result type must be orderable"); + FunctionRegistry functionRegistry = metadata.getFunctionRegistry(); + Symbol subValue = symbolAllocator.newSymbol(aggregationFunction.toString(), outputColumnType); + subqueryPlan = new AggregationNode( + idAllocator.getNextId(), + subqueryPlan, + ImmutableMap.of(subValue, new FunctionCall(aggregationFunction, ImmutableList.of(outputColumn.toSymbolReference()))), + ImmutableMap.of(subValue, functionRegistry.resolveFunction(aggregationFunction, ImmutableList.of(outputColumnType.getTypeSignature()))), + ImmutableMap.of(), + ImmutableList.of(ImmutableList.of()), + AggregationNode.Step.SINGLE, + Optional.empty(), + Optional.empty()); + subqueryPlan = new EnforceSingleRowNode(idAllocator.getNextId(), subqueryPlan); + ComparisonExpression valueComparedToSubquery = new ComparisonExpression(quantifiedComparison.getComparisonType(), quantifiedComparison.getValue(), subValue.toSymbolReference()); + subPlan.getTranslations().addIntermediateMapping(quantifiedComparison, valueComparedToSubquery); + return appendSubqueryApplyNode(subPlan, quantifiedComparison.getSubquery(), quantifiedComparison.getSubquery(), subqueryPlan, correlationAllowed); + } + + private static QualifiedName chooseAggregationFunction(QuantifiedComparisonExpression quantifiedComparison) + { + switch (quantifiedComparison.getQuantifier()) { + case ALL: + switch (quantifiedComparison.getComparisonType()) { + case LESS_THAN: + case LESS_THAN_OR_EQUAL: + return QualifiedName.of("min"); + case GREATER_THAN: + case GREATER_THAN_OR_EQUAL: + return QualifiedName.of("max"); + } + break; + case ANY: + case SOME: + switch (quantifiedComparison.getComparisonType()) { + case LESS_THAN: + case LESS_THAN_OR_EQUAL: + return QualifiedName.of("max"); + case GREATER_THAN: + case GREATER_THAN_OR_EQUAL: + return QualifiedName.of("min"); + } + break; + } + throw new IllegalArgumentException("Unexpected quantifier: " + quantifiedComparison.getQuantifier()); } - private static boolean isAggregationWithEmptyGroupBy(PlanNode subqueryPlan) + private boolean isAggregationWithEmptyGroupBy(PlanNode subqueryPlan) { return searchFrom(subqueryPlan) .skipOnlyWhen(Predicates.isInstanceOfAny(ProjectNode.class)) @@ -358,7 +468,7 @@ private static boolean isAggregationWithEmptyGroupBy(PlanNode subqueryPlan) .orElse(false); } - private PlanBuilder appendApplyNode(PlanBuilder subPlan, Node subquery, PlanNode subqueryNode, Map subqueryAssignments, boolean correlationAllowed) + private PlanBuilder appendSubqueryApplyNode(PlanBuilder subPlan, Expression subqueryExpression, Node subquery, PlanNode subqueryNode, boolean correlationAllowed) { Map correlation = extractCorrelation(subPlan, subqueryNode); if (!correlationAllowed && !correlation.isEmpty()) { @@ -368,20 +478,23 @@ private PlanBuilder appendApplyNode(PlanBuilder subPlan, Node subquery, PlanNode subqueryNode = replaceExpressionsWithSymbols(subqueryNode, correlation); TranslationMap translations = subPlan.copyTranslations(); - PlanNode root = subPlan.getRoot(); - return new PlanBuilder(translations, - new ApplyNode(idAllocator.getNextId(), - root, - subqueryNode, - subqueryAssignments, - ImmutableList.copyOf(correlation.values())), - analysis.getParameters()); - } + if (subqueryNode.getOutputSymbols().size() == 1) { + translations.put(subqueryExpression, subqueryNode.getOutputSymbols().get(0)); + } - private Map identityAssigments(PlanNode node) - { - return node.getOutputSymbols().stream() - .collect(toImmutableMap(symbol -> symbol, Symbol::toSymbolReference)); + PlanNode root = subPlan.getRoot(); + if (root.getOutputSymbols().isEmpty()) { + // there is nothing to join with - e.g. SELECT (SELECT 1) + return new PlanBuilder(translations, subqueryNode, analysis.getParameters()); + } + else { + return new PlanBuilder(translations, + new ApplyNode(idAllocator.getNextId(), + root, + subqueryNode, + ImmutableList.copyOf(correlation.values())), + analysis.getParameters()); + } } private Map extractCorrelation(PlanBuilder subPlan, PlanNode subquery) @@ -400,7 +513,7 @@ private Map extractCorrelation(PlanBuilder subPlan, PlanNode /** * Checks if give reference expression can resolved within given plan. */ - private static Optional tryResolveMissingExpression(PlanBuilder subPlan, Expression expression) + private Optional tryResolveMissingExpression(PlanBuilder subPlan, Expression expression) { Expression rewritten = subPlan.rewrite(expression); if (rewritten instanceof SymbolReference) { @@ -411,13 +524,13 @@ private static Optional tryResolveMissingExpression(PlanBuilder subP private RelationPlan createRelationPlan(Query subquery) { - return new RelationPlanner(analysis, symbolAllocator, idAllocator, lambdaDeclarationToSymbolMap, metadata, session) + return new RelationPlanner(analysis, symbolAllocator, idAllocator, metadata, session) .process(subquery, null); } private RelationPlan createRelationPlan(Expression subquery) { - return new RelationPlanner(analysis, symbolAllocator, idAllocator, lambdaDeclarationToSymbolMap, metadata, session) + return new RelationPlanner(analysis, symbolAllocator, idAllocator, metadata, session) .process(subquery, null); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/SymbolToInputRewriter.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/SymbolToInputRewriter.java index 74cef63f8de2..42f5cc8dc982 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/SymbolToInputRewriter.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/SymbolToInputRewriter.java @@ -17,7 +17,6 @@ import com.facebook.presto.sql.tree.ExpressionRewriter; import com.facebook.presto.sql.tree.ExpressionTreeRewriter; import com.facebook.presto.sql.tree.FieldReference; -import com.facebook.presto.sql.tree.LambdaExpression; import com.facebook.presto.sql.tree.SymbolReference; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; @@ -27,6 +26,7 @@ import static java.util.Objects.requireNonNull; public class SymbolToInputRewriter + extends ExpressionRewriter { private final Map symbolToChannelMapping; @@ -36,40 +36,12 @@ public SymbolToInputRewriter(Map symbolToChannelMapping) this.symbolToChannelMapping = ImmutableMap.copyOf(symbolToChannelMapping); } - public Expression rewrite(Expression expression) + @Override + public Expression rewriteSymbolReference(SymbolReference node, Void context, ExpressionTreeRewriter treeRewriter) { - return ExpressionTreeRewriter.rewriteWith(new ExpressionRewriter() { - @Override - public Expression rewriteSymbolReference(SymbolReference node, Context context, ExpressionTreeRewriter treeRewriter) - { - Integer channel = symbolToChannelMapping.get(Symbol.from(node)); - if (channel == null) { - Preconditions.checkArgument(context.isInLambda(), "Cannot resolve symbol %s", node.getName()); - return node; - } - return new FieldReference(channel); - } + Integer channel = symbolToChannelMapping.get(Symbol.from(node)); + Preconditions.checkArgument(channel != null, "Cannot resolve symbol %s", node.getName()); - @Override - public Expression rewriteLambdaExpression(LambdaExpression node, Context context, ExpressionTreeRewriter treeRewriter) - { - return treeRewriter.defaultRewrite(node, new Context(true)); - } - }, expression, new Context(false)); - } - - private static class Context - { - private final boolean inLambda; - - public Context(boolean inLambda) - { - this.inLambda = inLambda; - } - - public boolean isInLambda() - { - return inLambda; - } + return new FieldReference(channel); } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/TranslationMap.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/TranslationMap.java index eac911795929..4519d93b9d09 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/TranslationMap.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/TranslationMap.java @@ -22,13 +22,9 @@ import com.facebook.presto.sql.tree.ExpressionRewriter; import com.facebook.presto.sql.tree.ExpressionTreeRewriter; import com.facebook.presto.sql.tree.FieldReference; -import com.facebook.presto.sql.tree.LambdaArgumentDeclaration; -import com.facebook.presto.sql.tree.LambdaExpression; import com.facebook.presto.sql.tree.QualifiedNameReference; -import com.google.common.collect.ImmutableList; import java.util.HashMap; -import java.util.IdentityHashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -45,7 +41,6 @@ class TranslationMap // all expressions are rewritten in terms of fields declared by this relation plan private final RelationPlan rewriteBase; private final Analysis analysis; - private final IdentityHashMap lambdaDeclarationToSymbolMap; // current mappings of underlying field -> symbol for translating direct field references private final Symbol[] fieldSymbols; @@ -54,11 +49,10 @@ class TranslationMap private final Map expressionToSymbols = new HashMap<>(); private final Map expressionToExpressions = new HashMap<>(); - public TranslationMap(RelationPlan rewriteBase, Analysis analysis, IdentityHashMap lambdaDeclarationToSymbolMap) + public TranslationMap(RelationPlan rewriteBase, Analysis analysis) { this.rewriteBase = requireNonNull(rewriteBase, "rewriteBase is null"); this.analysis = requireNonNull(analysis, "analysis is null"); - this.lambdaDeclarationToSymbolMap = requireNonNull(lambdaDeclarationToSymbolMap, "lambdaDeclarationToSymbolMap is null"); fieldSymbols = new Symbol[rewriteBase.getOutputSymbols().size()]; } @@ -73,11 +67,6 @@ public Analysis getAnalysis() return analysis; } - public IdentityHashMap getLambdaDeclarationToSymbolMap() - { - return lambdaDeclarationToSymbolMap; - } - public void setFieldMappings(List symbols) { checkArgument(symbols.size() == fieldSymbols.length, "size of symbols list (%s) doesn't match number of expected fields (%s)", symbols.size(), fieldSymbols.length); @@ -238,14 +227,7 @@ public Expression rewriteFieldReference(FieldReference node, Void context, Expre @Override public Expression rewriteQualifiedNameReference(QualifiedNameReference node, Void context, ExpressionTreeRewriter treeRewriter) { - LambdaArgumentDeclaration referencedLambdaArgumentDeclaration = analysis.getLambdaArgumentReference(node); - if (referencedLambdaArgumentDeclaration != null) { - Symbol symbol = lambdaDeclarationToSymbolMap.get(referencedLambdaArgumentDeclaration); - return coerceIfNecessary(node, symbol.toSymbolReference()); - } - else { - return rewriteExpressionWithResolvedName(node); - } + return rewriteExpressionWithResolvedName(node); } private Expression rewriteExpressionWithResolvedName(Expression node) @@ -271,19 +253,6 @@ public Expression rewriteDereferenceExpression(DereferenceExpression node, Void return rewriteExpression(node, context, treeRewriter); } - @Override - public Expression rewriteLambdaExpression(LambdaExpression node, Void context, ExpressionTreeRewriter treeRewriter) - { - checkState(analysis.getCoercion(node) == null, "cannot coerce a lambda expression"); - - ImmutableList.Builder newArguments = ImmutableList.builder(); - for (LambdaArgumentDeclaration argument : node.getArguments()) { - newArguments.add(new LambdaArgumentDeclaration(lambdaDeclarationToSymbolMap.get(argument).getName())); - } - Expression rewrittenBody = treeRewriter.rewrite(node.getBody(), null); - return new LambdaExpression(newArguments.build(), rewrittenBody); - } - private Expression coerceIfNecessary(Expression original, Expression rewritten) { Type coercion = analysis.getCoercion(original); @@ -296,6 +265,6 @@ private Expression coerceIfNecessary(Expression original, Expression rewritten) } return rewritten; } - }, expression, null); + }, expression); } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddExchanges.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddExchanges.java index 49fc6e6e5d1e..86bac59425e7 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddExchanges.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/AddExchanges.java @@ -1160,7 +1160,6 @@ public PlanWithProperties visitApply(ApplyNode node, Context context) node.getId(), input.getNode(), subquery.getNode(), - node.getSubqueryAssignments(), node.getCorrelation()); return new PlanWithProperties(rewritten, deriveProperties(rewritten, ImmutableList.of(input.getProperties(), subquery.getProperties()))); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/CanonicalizeExpressions.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/CanonicalizeExpressions.java index f859d1e99be0..e89a38a6856a 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/CanonicalizeExpressions.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/CanonicalizeExpressions.java @@ -137,15 +137,15 @@ public Expression rewriteCurrentTime(CurrentTime node, Void context, ExpressionT switch (node.getType()) { case DATE: - return new FunctionCall(QualifiedName.of("current_date"), ImmutableList.of()); + return new FunctionCall(QualifiedName.of("current_date"), ImmutableList.of()); case TIME: - return new FunctionCall(QualifiedName.of("current_time"), ImmutableList.of()); + return new FunctionCall(QualifiedName.of("current_time"), ImmutableList.of()); case LOCALTIME: - return new FunctionCall(QualifiedName.of("localtime"), ImmutableList.of()); + return new FunctionCall(QualifiedName.of("localtime"), ImmutableList.of()); case TIMESTAMP: - return new FunctionCall(QualifiedName.of("current_timestamp"), ImmutableList.of()); + return new FunctionCall(QualifiedName.of("current_timestamp"), ImmutableList.of()); case LOCALTIMESTAMP: - return new FunctionCall(QualifiedName.of("localtimestamp"), ImmutableList.of()); + return new FunctionCall(QualifiedName.of("localtimestamp"), ImmutableList.of()); default: throw new UnsupportedOperationException("not yet implemented: " + node.getType()); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/EvaluateConstantApply.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/EvaluateConstantApply.java new file mode 100644 index 000000000000..6292e3d98516 --- /dev/null +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/EvaluateConstantApply.java @@ -0,0 +1,84 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.sql.planner.optimizations; + +import com.facebook.presto.Session; +import com.facebook.presto.spi.type.Type; +import com.facebook.presto.sql.planner.PlanNodeIdAllocator; +import com.facebook.presto.sql.planner.Symbol; +import com.facebook.presto.sql.planner.SymbolAllocator; +import com.facebook.presto.sql.planner.plan.ApplyNode; +import com.facebook.presto.sql.planner.plan.PlanNode; +import com.facebook.presto.sql.planner.plan.SimplePlanRewriter; +import com.facebook.presto.sql.planner.plan.ValuesNode; +import com.facebook.presto.sql.tree.Expression; +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.Map; + +/** + * Evaluates a constant Apply expression. For example: + *

+ * apply(values(a, b), r -> values(x, y)) + *

+ * into + *

+ * values((a, x), (a, y), (b, x), (b, y)) + */ +public class EvaluateConstantApply + implements PlanOptimizer +{ + @Override + public PlanNode optimize(PlanNode plan, Session session, Map types, SymbolAllocator symbolAllocator, PlanNodeIdAllocator idAllocator) + { + return SimplePlanRewriter.rewriteWith(new Rewriter(), plan); + } + + private static class Rewriter + extends SimplePlanRewriter + { + @Override + public PlanNode visitApply(ApplyNode node, RewriteContext context) + { + if (!node.getCorrelation().isEmpty()) { + // TODO: we should be able to support this by "evaluating" the + // expressions on the inner side after binding the values from + // the outer side + return context.defaultRewrite(node); + } + + if (!(node.getInput() instanceof ValuesNode) || !(node.getSubquery() instanceof ValuesNode)) { + return context.defaultRewrite(node); + } + + ValuesNode outer = (ValuesNode) node.getInput(); + ValuesNode inner = (ValuesNode) node.getSubquery(); + + // semantics of apply are similar to a cross join + ImmutableList.Builder> result = ImmutableList.builder(); + + for (List outerRow : outer.getRows()) { + for (List innerRow : inner.getRows()) { + result.add(ImmutableList.builder() + .addAll(outerRow) + .addAll(innerRow) + .build()); + } + } + + return new ValuesNode(node.getId(), node.getOutputSymbols(), result.build()); + } + } +} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ExpressionEquivalence.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ExpressionEquivalence.java index a1f43a1da191..61c50ad8eb96 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ExpressionEquivalence.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ExpressionEquivalence.java @@ -23,11 +23,10 @@ import com.facebook.presto.sql.relational.CallExpression; import com.facebook.presto.sql.relational.ConstantExpression; import com.facebook.presto.sql.relational.InputReferenceExpression; -import com.facebook.presto.sql.relational.LambdaDefinitionExpression; import com.facebook.presto.sql.relational.RowExpression; import com.facebook.presto.sql.relational.RowExpressionVisitor; -import com.facebook.presto.sql.relational.VariableReferenceExpression; import com.facebook.presto.sql.tree.Expression; +import com.facebook.presto.sql.tree.ExpressionTreeRewriter; import com.google.common.collect.ComparisonChain; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -97,7 +96,7 @@ public boolean areExpressionsEquivalent(Session session, Expression leftExpressi private RowExpression toRowExpression(Session session, Expression expression, Map symbolInput, Map inputTypes) { // replace qualified names with input references since row expressions do not support these - Expression expressionWithInputReferences = new SymbolToInputRewriter(symbolInput).rewrite(expression); + Expression expressionWithInputReferences = ExpressionTreeRewriter.rewriteWith(new SymbolToInputRewriter(symbolInput), expression); // determine the type of every expression IdentityHashMap expressionTypes = getExpressionTypesFromInput( @@ -205,18 +204,6 @@ public RowExpression visitInputReference(InputReferenceExpression node, Void con { return node; } - - @Override - public RowExpression visitLambda(LambdaDefinitionExpression lambda, Void context) - { - return new LambdaDefinitionExpression(lambda.getArgumentTypes(), lambda.getArguments(), lambda.getBody().accept(this, context)); - } - - @Override - public RowExpression visitVariableReference(VariableReferenceExpression reference, Void context) - { - return reference; - } } private static class RowExpressionComparator diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/HashGenerationOptimizer.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/HashGenerationOptimizer.java index 3180be3a8c13..df176fd034e4 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/HashGenerationOptimizer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/HashGenerationOptimizer.java @@ -484,7 +484,7 @@ public PlanWithProperties visitExchange(ExchangeNode node, HashComputationSet pa newSources.add(child.getNode()); // add hash symbols to inputs in the required order - ImmutableList.Builder newInputSymbols = ImmutableList.builder(); + ImmutableList.Builder newInputSymbols = ImmutableList.builder(); newInputSymbols.addAll(node.getInputs().get(sourceId)); for (HashComputation preferredHashSymbol : hashSymbolOrder) { HashComputation hashComputation = preferredHashSymbol.translate(outputToInputTranslator).get(); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ImplementSampleAsFilter.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ImplementSampleAsFilter.java index 4b0271102093..6663f4993bfd 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ImplementSampleAsFilter.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ImplementSampleAsFilter.java @@ -25,6 +25,7 @@ import com.facebook.presto.sql.tree.ComparisonExpression; import com.facebook.presto.sql.tree.ComparisonExpressionType; import com.facebook.presto.sql.tree.DoubleLiteral; +import com.facebook.presto.sql.tree.Expression; import com.facebook.presto.sql.tree.FunctionCall; import com.facebook.presto.sql.tree.QualifiedName; import com.google.common.collect.ImmutableList; @@ -59,7 +60,7 @@ public PlanNode visitSample(SampleNode node, RewriteContext context) ComparisonExpression expression = new ComparisonExpression( ComparisonExpressionType.LESS_THAN, - new FunctionCall(QualifiedName.of("rand"), ImmutableList.of()), + new FunctionCall(QualifiedName.of("rand"), ImmutableList.of()), new DoubleLiteral(Double.toString(node.getSampleRatio()))); return new FilterNode(node.getId(), rewrittenSource, expression); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/MergeWindows.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/MergeWindows.java index 57e5259023c6..123db7ecef1f 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/MergeWindows.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/MergeWindows.java @@ -65,7 +65,6 @@ public class MergeWindows implements PlanOptimizer { - @Override public PlanNode optimize(PlanNode plan, Session session, Map types, diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/MetadataDeleteOptimizer.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/MetadataDeleteOptimizer.java index 306fbb21c0e2..a697f44e5495 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/MetadataDeleteOptimizer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/MetadataDeleteOptimizer.java @@ -92,7 +92,7 @@ public PlanNode visitTableFinish(TableFinishNode node, RewriteContext cont return new MetadataDeleteNode(idAllocator.getNextId(), delete.get().getTarget(), Iterables.getOnlyElement(node.getOutputSymbols()), tableScanNode.getLayout().get()); } - private static Optional findNode(PlanNode source, Class clazz) + private Optional findNode(PlanNode source, Class clazz) { while (true) { // allow any chain of linear exchanges diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/MetadataQueryOptimizer.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/MetadataQueryOptimizer.java index b8942447149e..6a786c3fdd42 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/MetadataQueryOptimizer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/MetadataQueryOptimizer.java @@ -133,7 +133,7 @@ public PlanNode visitAggregation(AggregationNode node, RewriteContext cont // with a Values node TableLayout layout = null; if (!tableScan.getLayout().isPresent()) { - List layouts = metadata.getLayouts(session, tableScan.getTable(), Constraint.alwaysTrue(), Optional.empty()); + List layouts = metadata.getLayouts(session, tableScan.getTable(), Constraint.alwaysTrue(), Optional.empty()); if (layouts.size() == 1) { layout = Iterables.getOnlyElement(layouts).getLayout(); } @@ -180,7 +180,7 @@ public PlanNode visitAggregation(AggregationNode node, RewriteContext cont return SimplePlanRewriter.rewriteWith(new Replacer(valuesNode), node); } - private static Optional findTableScan(PlanNode source) + private Optional findTableScan(PlanNode source) { while (true) { // allow any chain of linear transformations diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/OptimizeMixedDistinctAggregations.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/OptimizeMixedDistinctAggregations.java index 6cfdd33bdbac..bb0e867ecba8 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/OptimizeMixedDistinctAggregations.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/OptimizeMixedDistinctAggregations.java @@ -18,7 +18,6 @@ import com.facebook.presto.metadata.Signature; import com.facebook.presto.spi.type.BigintType; import com.facebook.presto.spi.type.Type; -import com.facebook.presto.sql.analyzer.TypeSignatureProvider; import com.facebook.presto.sql.planner.PlanNodeIdAllocator; import com.facebook.presto.sql.planner.Symbol; import com.facebook.presto.sql.planner.SymbolAllocator; @@ -170,10 +169,10 @@ public PlanNode visitAggregation(AggregationNode node, RewriteContextemptyMap(), node.getGroupingSets(), node.getStep(), - Optional.empty(), + Optional.empty(), node.getGroupIdSymbol()); } @@ -419,7 +418,7 @@ private AggregationNode createNonDistinctAggregation( groupIdNode, aggregations.build(), functions.build(), - Collections.emptyMap(), + Collections.emptyMap(), ImmutableList.of(groupByKeys), SINGLE, originalNode.getHashSymbol(), @@ -431,11 +430,11 @@ private Signature getFunctionSignature(QualifiedName functionName, Symbol argume return metadata.getFunctionRegistry() .resolveFunction( functionName, - ImmutableList.of(new TypeSignatureProvider(symbolAllocator.getTypes().get(argument).getTypeSignature()))); + ImmutableList.of(symbolAllocator.getTypes().get(argument).getTypeSignature())); } // creates if clause specific to use case here, default value always null - private static IfExpression createIfExpression(Expression left, Expression right, ComparisonExpressionType type, Expression result, Type trueValueType) + private IfExpression createIfExpression(Expression left, Expression right, ComparisonExpressionType type, Expression result, Type trueValueType) { return new IfExpression( new ComparisonExpression(type, left, right), diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PredicatePushDown.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PredicatePushDown.java index 5ab3d31b377b..1af948a65a49 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PredicatePushDown.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PredicatePushDown.java @@ -82,6 +82,7 @@ import static com.facebook.presto.sql.planner.plan.JoinNode.Type.INNER; import static com.facebook.presto.sql.planner.plan.JoinNode.Type.LEFT; import static com.facebook.presto.sql.planner.plan.JoinNode.Type.RIGHT; +import static com.facebook.presto.util.ImmutableCollectors.toImmutableList; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import static com.google.common.base.Predicates.equalTo; @@ -378,15 +379,10 @@ public PlanNode visitJoin(JoinNode node, RewriteContext context) Expression leftExpression = (alignedComparison) ? equality.getLeft() : equality.getRight(); Expression rightExpression = (alignedComparison) ? equality.getRight() : equality.getLeft(); - Symbol leftSymbol = symbolForExpression(leftExpression); - if (!node.getLeft().getOutputSymbols().contains(leftSymbol)) { - leftProjections.put(leftSymbol, leftExpression); - } - - Symbol rightSymbol = symbolForExpression(rightExpression); - if (!node.getRight().getOutputSymbols().contains(rightSymbol)) { - rightProjections.put(rightSymbol, rightExpression); - } + Symbol leftSymbol = symbolAllocator.newSymbol(leftExpression, extractType(leftExpression)); + leftProjections.put(leftSymbol, leftExpression); + Symbol rightSymbol = symbolAllocator.newSymbol(rightExpression, extractType(rightExpression)); + rightProjections.put(rightSymbol, rightExpression); joinConditionBuilder.add(new JoinNode.EquiJoinClause(leftSymbol, rightSymbol)); } @@ -411,16 +407,7 @@ public PlanNode visitJoin(JoinNode node, RewriteContext context) return output; } - private Symbol symbolForExpression(Expression expression) - { - if (expression instanceof SymbolReference) { - return Symbol.from(expression); - } - - return symbolAllocator.newSymbol(expression, extractType(expression)); - } - - private static OuterJoinPushDownResult processLimitedOuterJoin(Expression inheritedPredicate, Expression outerEffectivePredicate, Expression innerEffectivePredicate, Expression joinPredicate, Collection outerSymbols) + private OuterJoinPushDownResult processLimitedOuterJoin(Expression inheritedPredicate, Expression outerEffectivePredicate, Expression innerEffectivePredicate, Expression joinPredicate, Collection outerSymbols) { checkArgument(Iterables.all(DependencyExtractor.extractUnique(outerEffectivePredicate), in(outerSymbols)), "outerEffectivePredicate must only contain symbols from outerSymbols"); checkArgument(Iterables.all(DependencyExtractor.extractUnique(innerEffectivePredicate), not(in(outerSymbols))), "innerEffectivePredicate must not contain symbols from outerSymbols"); @@ -541,7 +528,7 @@ private Expression getPostJoinPredicate() } } - private static InnerJoinPushDownResult processInnerJoin(Expression inheritedPredicate, Expression leftEffectivePredicate, Expression rightEffectivePredicate, Expression joinPredicate, Collection leftSymbols) + private InnerJoinPushDownResult processInnerJoin(Expression inheritedPredicate, Expression leftEffectivePredicate, Expression rightEffectivePredicate, Expression joinPredicate, Collection leftSymbols) { checkArgument(Iterables.all(DependencyExtractor.extractUnique(leftEffectivePredicate), in(leftSymbols)), "leftEffectivePredicate must only contain symbols from leftSymbols"); checkArgument(Iterables.all(DependencyExtractor.extractUnique(rightEffectivePredicate), not(in(leftSymbols))), "rightEffectivePredicate must not contain symbols from leftSymbols"); @@ -621,7 +608,18 @@ private static InnerJoinPushDownResult processInnerJoin(Expression inheritedPred rightPushDownConjuncts.addAll(allInferenceWithoutRightInferred.generateEqualitiesPartitionedBy(not(in(leftSymbols))).getScopeEqualities()); joinConjuncts.addAll(allInference.generateEqualitiesPartitionedBy(in(leftSymbols)).getScopeStraddlingEqualities()); // scope straddling equalities get dropped in as part of the join predicate - return new InnerJoinPushDownResult(combineConjuncts(leftPushDownConjuncts.build()), combineConjuncts(rightPushDownConjuncts.build()), combineConjuncts(joinConjuncts.build()), BooleanLiteral.TRUE_LITERAL); + // Since we only currently support equality in join conjuncts, factor out the non-equality conjuncts to a post-join filter + List joinConjunctsList = joinConjuncts.build(); + + List postJoinConjuncts = joinConjunctsList.stream() + .filter(joinEqualityExpression(leftSymbols).negate()) + .collect(toImmutableList()); + + joinConjunctsList = joinConjunctsList.stream() + .filter(joinEqualityExpression(leftSymbols)) + .collect(toImmutableList()); + + return new InnerJoinPushDownResult(combineConjuncts(leftPushDownConjuncts.build()), combineConjuncts(rightPushDownConjuncts.build()), combineConjuncts(joinConjunctsList), combineConjuncts(postJoinConjuncts)); } private static class InnerJoinPushDownResult diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PreferredProperties.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PreferredProperties.java index ba5df3954db6..b46dfa071ed1 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PreferredProperties.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PreferredProperties.java @@ -230,7 +230,7 @@ public static Global distributed(Optional partitioningPr public static Global distributed() { - return distributed(Optional.empty()); + return distributed(Optional.empty()); } public static Global distributed(PartitioningProperties partitioning) diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PropertyDerivations.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PropertyDerivations.java index f7e0b115e663..8c8f3537df46 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PropertyDerivations.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PropertyDerivations.java @@ -244,7 +244,7 @@ public ActualProperties visitAggregation(AggregationNode node, List node.getGroupingKeys().contains(symbol) ? Optional.of(symbol) : Optional.empty()); + ActualProperties translated = properties.translate(symbol -> node.getGroupingKeys().contains(symbol) ? Optional.of(symbol) : Optional.empty()); return ActualProperties.builderFrom(translated) .local(LocalProperties.grouped(node.getGroupingKeys())) diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PruneUnreferencedOutputs.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PruneUnreferencedOutputs.java index 71abc846008f..95d53cf54835 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PruneUnreferencedOutputs.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/PruneUnreferencedOutputs.java @@ -18,6 +18,7 @@ import com.facebook.presto.spi.ColumnHandle; import com.facebook.presto.spi.type.Type; import com.facebook.presto.sql.planner.DependencyExtractor; +import com.facebook.presto.sql.planner.ExpressionExtractor; import com.facebook.presto.sql.planner.PartitioningScheme; import com.facebook.presto.sql.planner.PlanNodeIdAllocator; import com.facebook.presto.sql.planner.Symbol; @@ -42,7 +43,6 @@ import com.facebook.presto.sql.planner.plan.ProjectNode; import com.facebook.presto.sql.planner.plan.RowNumberNode; import com.facebook.presto.sql.planner.plan.SemiJoinNode; -import com.facebook.presto.sql.planner.plan.SetOperationNode; import com.facebook.presto.sql.planner.plan.SimplePlanRewriter; import com.facebook.presto.sql.planner.plan.SortNode; import com.facebook.presto.sql.planner.plan.TableFinishNode; @@ -56,6 +56,7 @@ import com.facebook.presto.sql.planner.plan.WindowNode; import com.facebook.presto.sql.tree.Expression; import com.facebook.presto.sql.tree.FunctionCall; +import com.facebook.presto.sql.tree.SymbolReference; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableListMultimap; import com.google.common.collect.ImmutableMap; @@ -74,12 +75,12 @@ import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; +import java.util.stream.Stream; import static com.facebook.presto.util.ImmutableCollectors.toImmutableList; import static com.facebook.presto.util.ImmutableCollectors.toImmutableSet; import static com.google.common.base.Predicates.in; import static com.google.common.collect.Iterables.concat; -import static com.google.common.collect.Sets.intersection; import static java.util.Objects.requireNonNull; /** @@ -149,7 +150,7 @@ public PlanNode visitExchange(ExchangeNode node, RewriteContext> con node.getPartitioningScheme().isReplicateNulls(), node.getPartitioningScheme().getBucketToPartition()); - ImmutableList.Builder rewrittenSources = ImmutableList.builder(); + ImmutableList.Builder rewrittenSources = ImmutableList.builder(); for (int i = 0; i < node.getSources().size(); i++) { ImmutableSet.Builder expectedInputs = ImmutableSet.builder() .addAll(inputsBySource.get(i)); @@ -483,6 +484,7 @@ public PlanNode visitProject(ProjectNode node, RewriteContext> conte ImmutableSet.Builder expectedInputs = ImmutableSet.builder(); ImmutableMap.Builder builder = ImmutableMap.builder(); + ImmutableList.Builder removedExpressions = ImmutableList.builder(); for (int i = 0; i < node.getOutputSymbols().size(); i++) { Symbol output = node.getOutputSymbols().get(i); Expression expression = node.getAssignments().get(output); @@ -491,11 +493,35 @@ public PlanNode visitProject(ProjectNode node, RewriteContext> conte expectedInputs.addAll(DependencyExtractor.extractUnique(expression)); builder.put(output, expression); } + else { + removedExpressions.add(expression); + } } - PlanNode source = context.rewrite(node.getSource(), expectedInputs.build()); + Map assignments = builder.build(); + PlanNode rewrittenSource = pruneUnreferencedApplyNodes(removedExpressions.build(), node.getSource(), assignments); + rewrittenSource = context.rewrite(rewrittenSource, expectedInputs.build()); + + return new ProjectNode(node.getId(), rewrittenSource, assignments); + } + + private PlanNode pruneUnreferencedApplyNodes(List removedExpressions, PlanNode node, Map assignments) + { + Set symbolsUsedByProjection = assignments.values().stream() + .map(DependencyExtractor::extractUnique) + .flatMap(Set::stream) + .collect(toImmutableSet()); - return new ProjectNode(node.getId(), source, builder.build()); + PlanNode rewrittenNode = node; + for (Expression removedExpression : removedExpressions) { + for (Symbol symbol : DependencyExtractor.extractUnique(removedExpression)) { + if (!symbolsUsedByProjection.contains(symbol)) { + UnusedApplyRemover unusedApplyRemover = new UnusedApplyRemover(symbol.toSymbolReference()); + rewrittenNode = SimplePlanRewriter.rewriteWith(unusedApplyRemover, rewrittenNode, null); + } + } + } + return rewrittenNode; } @Override @@ -629,29 +655,6 @@ public PlanNode visitDelete(DeleteNode node, RewriteContext> context @Override public PlanNode visitUnion(UnionNode node, RewriteContext> context) - { - ListMultimap rewrittenSymbolMapping = rewriteSetOperationSymbolMapping(node, context); - ImmutableList rewrittenSubPlans = rewriteSetOperationSubPlans(node, context, rewrittenSymbolMapping); - return new UnionNode(node.getId(), rewrittenSubPlans, rewrittenSymbolMapping, ImmutableList.copyOf(rewrittenSymbolMapping.keySet())); - } - - @Override - public PlanNode visitIntersect(IntersectNode node, RewriteContext> context) - { - ListMultimap rewrittenSymbolMapping = rewriteSetOperationSymbolMapping(node, context); - ImmutableList rewrittenSubPlans = rewriteSetOperationSubPlans(node, context, rewrittenSymbolMapping); - return new IntersectNode(node.getId(), rewrittenSubPlans, rewrittenSymbolMapping, ImmutableList.copyOf(rewrittenSymbolMapping.keySet())); - } - - @Override - public PlanNode visitExcept(ExceptNode node, RewriteContext> context) - { - ListMultimap rewrittenSymbolMapping = rewriteSetOperationSymbolMapping(node, context); - ImmutableList rewrittenSubPlans = rewriteSetOperationSubPlans(node, context, rewrittenSymbolMapping); - return new ExceptNode(node.getId(), rewrittenSubPlans, rewrittenSymbolMapping, ImmutableList.copyOf(rewrittenSymbolMapping.keySet())); - } - - private ListMultimap rewriteSetOperationSymbolMapping(SetOperationNode node, RewriteContext> context) { // Find out which output symbols we need to keep ImmutableListMultimap.Builder rewrittenSymbolMappingBuilder = ImmutableListMultimap.builder(); @@ -660,11 +663,8 @@ private ListMultimap rewriteSetOperationSymbolMapping(SetOperati rewrittenSymbolMappingBuilder.putAll(symbol, node.getSymbolMapping().get(symbol)); } } - return rewrittenSymbolMappingBuilder.build(); - } + ListMultimap rewrittenSymbolMapping = rewrittenSymbolMappingBuilder.build(); - private ImmutableList rewriteSetOperationSubPlans(SetOperationNode node, RewriteContext> context, ListMultimap rewrittenSymbolMapping) - { // Find the corresponding input symbol to the remaining output symbols and prune the subplans ImmutableList.Builder rewrittenSubPlans = ImmutableList.builder(); for (int i = 0; i < node.getSources().size(); i++) { @@ -674,7 +674,20 @@ private ImmutableList rewriteSetOperationSubPlans(SetOperationNode nod } rewrittenSubPlans.add(context.rewrite(node.getSources().get(i), expectedInputSymbols.build())); } - return rewrittenSubPlans.build(); + + return new UnionNode(node.getId(), rewrittenSubPlans.build(), rewrittenSymbolMapping, ImmutableList.copyOf(rewrittenSymbolMapping.keySet())); + } + + @Override + public PlanNode visitIntersect(IntersectNode node, RewriteContext> context) + { + return new IntersectNode(node.getId(), node.getSources(), node.getSymbolMapping(), ImmutableList.copyOf(node.getSymbolMapping().keySet())); + } + + @Override + public PlanNode visitExcept(ExceptNode node, RewriteContext> context) + { + return new ExceptNode(node.getId(), node.getSources(), node.getSymbolMapping(), ImmutableList.copyOf(node.getSymbolMapping().keySet())); } @Override @@ -707,25 +720,7 @@ public PlanNode visitValues(ValuesNode node, RewriteContext> context @Override public PlanNode visitApply(ApplyNode node, RewriteContext> context) { - // remove unused apply nodes - if (intersection(node.getSubqueryAssignments().keySet(), context.get()).isEmpty()) { - return context.rewrite(node.getInput(), context.get()); - } - - // extract symbols required subquery plan - ImmutableSet.Builder subqueryAssignmentsSymbolsBuilder = ImmutableSet.builder(); - ImmutableMap.Builder subqueryAssignments = ImmutableMap.builder(); - for (Map.Entry entry : node.getSubqueryAssignments().entrySet()) { - Symbol output = entry.getKey(); - Expression expression = entry.getValue(); - if (context.get().contains(output)) { - subqueryAssignmentsSymbolsBuilder.addAll(DependencyExtractor.extractUnique(expression)); - subqueryAssignments.put(output, expression); - } - } - - Set subqueryAssignmentsSymbols = subqueryAssignmentsSymbolsBuilder.build(); - PlanNode subquery = context.rewrite(node.getSubquery(), subqueryAssignmentsSymbols); + PlanNode subquery = context.rewrite(node.getSubquery(), context.get()); // prune not used correlation symbols Set subquerySymbols = DependencyExtractor.extractUnique(subquery); @@ -736,10 +731,107 @@ public PlanNode visitApply(ApplyNode node, RewriteContext> context) Set inputContext = ImmutableSet.builder() .addAll(context.get()) .addAll(newCorrelation) - .addAll(subqueryAssignmentsSymbols) // need to include those: e.g: "expr" from "expr IN (SELECT 1)" .build(); PlanNode input = context.rewrite(node.getInput(), inputContext); - return new ApplyNode(node.getId(), input, subquery, subqueryAssignments.build(), newCorrelation); + return new ApplyNode(node.getId(), input, subquery, newCorrelation); + } + } + + /** + * Removes ApplyNode which subquery produces given Expression (valueList of InPredicate or SymbolReference) only if + * that Expression is not used. + */ + private static class UnusedApplyRemover + extends ApplyNodeRewriter + { + public UnusedApplyRemover(SymbolReference symbolReference) + { + super(symbolReference); } + + @Override + protected PlanNode visitPlan(PlanNode node, RewriteContext context) + { + if (usesSymbol(node, getReferenceSymbol())) { + return node; + } + return context.defaultRewrite(node); + } + + @Override + public PlanNode visitProject(ProjectNode node, RewriteContext context) + { + return visitPlan(node, context); + } + + @Override + public PlanNode visitJoin(JoinNode node, RewriteContext context) + { + if (usesSymbol(node, getReferenceSymbol())) { + return node; + } + + boolean usesSymbolInCriteria = node.getCriteria().stream() + .flatMap(criteria -> Stream.of(criteria.getLeft(), criteria.getRight())) + .anyMatch(criteriaSymbol -> criteriaSymbol.equals(getReferenceSymbol())); + + if (usesSymbolInCriteria) { + return node; + } + + return context.defaultRewrite(node); + } + + @Override + public PlanNode visitIndexJoin(IndexJoinNode node, RewriteContext context) + { + if (usesSymbol(node, getReferenceSymbol())) { + return node; + } + + boolean usesSymbolInCriteria = node.getCriteria().stream() + .flatMap(criteria -> Stream.of(criteria.getProbe(), criteria.getIndex())) + .anyMatch(criteriaSymbol -> criteriaSymbol.equals(getReferenceSymbol())); + + if (usesSymbolInCriteria) { + return node; + } + + return context.defaultRewrite(node); + } + + private Symbol getReferenceSymbol() + { + return Symbol.from(reference); + } + + @Override + public PlanNode visitSemiJoin(SemiJoinNode node, RewriteContext context) + { + if (usesSymbol(node, getReferenceSymbol())) { + return node; + } + + boolean usesSymbolInCriteria = node.getSourceJoinSymbol().equals(getReferenceSymbol()) + && node.getFilteringSourceJoinSymbol().equals(getReferenceSymbol()); + + if (usesSymbolInCriteria) { + return node; + } + + return context.defaultRewrite(node); + } + + @Override + protected PlanNode rewriteApply(ApplyNode node) + { + return node.getInput(); + } + } + + private static boolean usesSymbol(PlanNode node, Symbol symbol) + { + return ExpressionExtractor.extractExpressionsNonRecursive(node).stream() + .anyMatch(nodeExpression -> DependencyExtractor.extractUnique(nodeExpression).contains(symbol)); } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/RemoveUnreferencedScalarInputApplyNodes.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/RemoveRedundantApply.java similarity index 58% rename from presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/RemoveUnreferencedScalarInputApplyNodes.java rename to presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/RemoveRedundantApply.java index 359e61ba352c..aa3bf0fd1955 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/RemoveUnreferencedScalarInputApplyNodes.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/RemoveRedundantApply.java @@ -1,4 +1,3 @@ - /* * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,34 +19,52 @@ import com.facebook.presto.sql.planner.Symbol; import com.facebook.presto.sql.planner.SymbolAllocator; import com.facebook.presto.sql.planner.plan.ApplyNode; +import com.facebook.presto.sql.planner.plan.EnforceSingleRowNode; import com.facebook.presto.sql.planner.plan.PlanNode; import com.facebook.presto.sql.planner.plan.SimplePlanRewriter; +import com.facebook.presto.sql.planner.plan.ValuesNode; import java.util.Map; -import static com.facebook.presto.sql.planner.optimizations.ScalarQueryUtil.isScalar; -import static com.facebook.presto.sql.planner.plan.SimplePlanRewriter.rewriteWith; - /** - * Remove resolved ApplyNodes with unreferenced scalar input, e.g: "SELECT (SELECT 1)". + * Removes unnecessary apply nodes of the form + *

+ * apply(x, r -> ()) + *

+ * or + *

+ * apply(x, r -> scalar(...)), where the scalar subquery produces no columns + *

+ * by rewriting them to x */ -public class RemoveUnreferencedScalarInputApplyNodes +public class RemoveRedundantApply implements PlanOptimizer { @Override public PlanNode optimize(PlanNode plan, Session session, Map types, SymbolAllocator symbolAllocator, PlanNodeIdAllocator idAllocator) { - return rewriteWith(new Rewriter(), plan, null); + return SimplePlanRewriter.rewriteWith(new Rewriter(), plan); } private static class Rewriter - extends SimplePlanRewriter + extends SimplePlanRewriter { @Override - public PlanNode visitApply(ApplyNode node, RewriteContext context) + public PlanNode visitApply(ApplyNode node, RewriteContext context) { - if (node.getInput().getOutputSymbols().isEmpty() && isScalar(node.getInput()) && node.isResolvedScalarSubquery()) { - return context.rewrite(node.getSubquery()); + // if the subquery produces no columns... + if (node.getSubquery().getOutputSymbols().isEmpty()) { + // and it's guaranteed to produce a single row + if (node.getSubquery() instanceof EnforceSingleRowNode) { + return node.getInput(); + } + } + + // or it's a VALUES with a single row + if ((node.getSubquery() instanceof ValuesNode)) { + if (((ValuesNode) node.getSubquery()).getRows().size() == 1) { + return node.getInput(); + } } return context.defaultRewrite(node); diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ScalarQueryUtil.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ScalarQueryUtil.java index cb8d0397dc91..cb1fc8dae2fb 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ScalarQueryUtil.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/ScalarQueryUtil.java @@ -13,15 +13,12 @@ */ package com.facebook.presto.sql.planner.optimizations; -import com.facebook.presto.sql.planner.plan.AggregationNode; import com.facebook.presto.sql.planner.plan.EnforceSingleRowNode; import com.facebook.presto.sql.planner.plan.ExchangeNode; import com.facebook.presto.sql.planner.plan.FilterNode; import com.facebook.presto.sql.planner.plan.PlanNode; import com.facebook.presto.sql.planner.plan.PlanVisitor; import com.facebook.presto.sql.planner.plan.ProjectNode; -import com.facebook.presto.sql.planner.plan.ValuesNode; -import com.google.common.collect.ImmutableList; import static com.google.common.collect.Iterables.getOnlyElement; @@ -49,12 +46,6 @@ public Boolean visitEnforceSingleRow(EnforceSingleRowNode node, Void context) return true; } - @Override - public Boolean visitAggregation(AggregationNode node, Void context) - { - return node.getGroupingSets().equals(ImmutableList.of(ImmutableList.of())); - } - @Override public Boolean visitExchange(ExchangeNode node, Void context) { @@ -73,10 +64,5 @@ public Boolean visitFilter(FilterNode node, Void context) { return node.getSource().accept(this, null); } - - public Boolean visitValues(ValuesNode node, Void context) - { - return node.getRows().size() == 1; - } } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/SetFlatteningOptimizer.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/SetFlatteningOptimizer.java index 2e969d827afc..1820daf7e37a 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/SetFlatteningOptimizer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/SetFlatteningOptimizer.java @@ -89,16 +89,15 @@ public PlanNode visitExcept(ExceptNode node, RewriteContext context) return new ExceptNode(node.getId(), flattenedSources.build(), flattenedSymbolMap.build(), ImmutableList.copyOf(flattenedSymbolMap.build().keySet())); } - private static void flattenSetOperation(SetOperationNode node, RewriteContext context, ImmutableList.Builder flattenedSources, ImmutableListMultimap.Builder flattenedSymbolMap) + private void flattenSetOperation(SetOperationNode node, RewriteContext context, ImmutableList.Builder flattenedSources, ImmutableListMultimap.Builder flattenedSymbolMap) { for (int i = 0; i < node.getSources().size(); i++) { PlanNode subplan = node.getSources().get(i); PlanNode rewrittenSource = context.rewrite(subplan, context.get()); Class setOperationClass = node.getClass(); - if (setOperationClass.isInstance(rewrittenSource) && (!setOperationClass.equals(ExceptNode.class) || i == 0)) { + if (setOperationClass.isInstance(rewrittenSource)) { // Absorb source's subplans if it is also a SetOperation of the same type - // ExceptNodes can only flatten their first source because except is not associative SetOperationNode rewrittenSetOperation = (SetOperationNode) rewrittenSource; flattenedSources.addAll(rewrittenSetOperation.getSources()); for (Map.Entry> entry : node.getSymbolMapping().asMap().entrySet()) { diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/SimplifyExpressions.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/SimplifyExpressions.java index 712768bc2708..1f07f01e54dc 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/SimplifyExpressions.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/SimplifyExpressions.java @@ -222,7 +222,7 @@ public Expression rewriteLogicalBinaryExpression(LogicalBinaryExpression node, N List> subPredicates = getSubPredicates(predicates); Set commonPredicates = ImmutableSet.copyOf(subPredicates.stream() - .map(ExtractCommonPredicatesExpressionRewriter::filterDeterministicPredicates) + .map(this::filterDeterministicPredicates) .reduce(Sets::intersection) .orElse(emptySet())); @@ -249,7 +249,7 @@ public Expression rewriteLogicalBinaryExpression(LogicalBinaryExpression node, N .build()); } - private static List> getSubPredicates(List predicates) + private List> getSubPredicates(List predicates) { return predicates.stream() .map(predicate -> predicate instanceof LogicalBinaryExpression ? @@ -257,7 +257,7 @@ private static List> getSubPredicates(List predicat .collect(toImmutableList()); } - private static Set filterDeterministicPredicates(List predicates) + private Set filterDeterministicPredicates(List predicates) { return predicates.stream() .filter(DeterminismEvaluator::isDeterministic) diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TransformCorrelatedScalarAggregationToJoin.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TransformCorrelatedScalarAggregationToJoin.java index efc68a3a0d56..30e043bb1a27 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TransformCorrelatedScalarAggregationToJoin.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TransformCorrelatedScalarAggregationToJoin.java @@ -55,7 +55,6 @@ import java.util.function.Predicate; import java.util.stream.Collectors; -import static com.facebook.presto.sql.analyzer.TypeSignatureProvider.fromTypeSignatures; import static com.facebook.presto.sql.planner.optimizations.PlanNodeSearcher.searchFrom; import static com.facebook.presto.sql.planner.optimizations.Predicates.isInstanceOfAny; import static com.facebook.presto.sql.planner.plan.SimplePlanRewriter.rewriteWith; @@ -131,7 +130,7 @@ public Rewriter(PlanNodeIdAllocator idAllocator, SymbolAllocator symbolAllocator public PlanNode visitApply(ApplyNode node, RewriteContext context) { ApplyNode rewrittenNode = (ApplyNode) context.defaultRewrite(node, context.get()); - if (!rewrittenNode.getCorrelation().isEmpty() && rewrittenNode.isResolvedScalarSubquery()) { + if (!rewrittenNode.getCorrelation().isEmpty()) { Optional aggregation = searchFrom(rewrittenNode.getSubquery()) .where(AggregationNode.class::isInstance) .skipOnlyWhen(isInstanceOfAny(ProjectNode.class, EnforceSingleRowNode.class)) @@ -238,9 +237,7 @@ private Optional createAggregationNode( ImmutableList.of(nonNullableAggregationSourceSymbol.toSymbolReference()))); List scalarAggregationSourceTypeSignatures = ImmutableList.of( symbolAllocator.getTypes().get(nonNullableAggregationSourceSymbol).getTypeSignature()); - functions.put(symbol, functionRegistry.resolveFunction( - count, - fromTypeSignatures(scalarAggregationSourceTypeSignatures))); + functions.put(symbol, functionRegistry.resolveFunction(count, scalarAggregationSourceTypeSignatures)); } else { aggregations.put(symbol, entry.getValue()); @@ -304,7 +301,7 @@ private Optional decorrelateFilters(PlanNode node, List decorrelatedNode( + private Optional decorrelatedNode( List correlatedPredicates, PlanNode node, List correlation) @@ -316,7 +313,7 @@ private static Optional decorrelatedNode( return Optional.of(new DecorrelatedNode(correlatedPredicates, node)); } - private static Predicate isUsingPredicate(List symbols) + private Predicate isUsingPredicate(List symbols) { return expression -> symbols.stream().anyMatch(DependencyExtractor.extractUnique(expression)::contains); } @@ -334,7 +331,7 @@ private PlanNode updateFilterNode(PlanNodeSearcher filterNodeSearcher, List - * - Project($0 > 0) - * - Aggregation(COUNT(*)) - * - Limit(1) - * -- subquery - * - */ -public class TransformExistsApplyToScalarApply - implements PlanOptimizer -{ - private final Metadata metadata; - - public TransformExistsApplyToScalarApply(Metadata metadata) - { - this.metadata = requireNonNull(metadata, "metadata is null"); - } - - @Override - public PlanNode optimize(PlanNode plan, Session session, Map types, SymbolAllocator symbolAllocator, PlanNodeIdAllocator idAllocator) - { - return rewriteWith(new Rewriter(idAllocator, symbolAllocator, metadata), plan, null); - } - - private static class Rewriter - extends SimplePlanRewriter - - { - private final PlanNodeIdAllocator idAllocator; - private final SymbolAllocator symbolAllocator; - private final Metadata metadata; - - public Rewriter(PlanNodeIdAllocator idAllocator, SymbolAllocator symbolAllocator, Metadata metadata) - { - this.idAllocator = requireNonNull(idAllocator, "idAllocator is null"); - this.symbolAllocator = requireNonNull(symbolAllocator, "symbolAllocator is null"); - this.metadata = requireNonNull(metadata, "metadata is null"); - } - - @Override - public PlanNode visitApply(ApplyNode node, RewriteContext context) - { - if (node.getSubqueryAssignments().size() != 1) { - return context.defaultRewrite(node); - } - - Expression expression = getOnlyElement(node.getSubqueryAssignments().values()); - if (!(expression instanceof ExistsPredicate)) { - return context.defaultRewrite(node); - } - - PlanNode input = context.rewrite(node.getInput()); - PlanNode subquery = context.rewrite(node.getSubquery()); - - subquery = new LimitNode(idAllocator.getNextId(), subquery, 1, false); - - FunctionRegistry functionRegistry = metadata.getFunctionRegistry(); - QualifiedName countFunction = QualifiedName.of("count"); - Symbol count = symbolAllocator.newSymbol(countFunction.toString(), BIGINT); - subquery = new AggregationNode( - idAllocator.getNextId(), - subquery, - ImmutableMap.of(count, new FunctionCall(countFunction, ImmutableList.of())), - ImmutableMap.of(count, functionRegistry.resolveFunction(countFunction, ImmutableList.of())), - ImmutableMap.of(), - ImmutableList.of(ImmutableList.of()), - AggregationNode.Step.SINGLE, - Optional.empty(), - Optional.empty()); - - ComparisonExpression countGreaterThanZero = new ComparisonExpression(GREATER_THAN, count.toSymbolReference(), new Cast(new LongLiteral("0"), BIGINT.toString())); - - Symbol existsSymbol = getOnlyElement(node.getSubqueryAssignments().keySet()); - subquery = new ProjectNode( - idAllocator.getNextId(), - subquery, - ImmutableMap.of(existsSymbol, countGreaterThanZero)); - - return new ApplyNode( - node.getId(), - input, - subquery, - ImmutableMap.of(existsSymbol, existsSymbol.toSymbolReference()), - node.getCorrelation()); - } - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TransformQuantifiedComparisonApplyToScalarApply.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TransformQuantifiedComparisonApplyToScalarApply.java deleted file mode 100644 index c1d86afeb633..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TransformQuantifiedComparisonApplyToScalarApply.java +++ /dev/null @@ -1,243 +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.sql.planner.optimizations; - -import com.facebook.presto.Session; -import com.facebook.presto.metadata.FunctionRegistry; -import com.facebook.presto.metadata.Metadata; -import com.facebook.presto.spi.type.Type; -import com.facebook.presto.spi.type.TypeSignature; -import com.facebook.presto.sql.analyzer.TypeSignatureProvider; -import com.facebook.presto.sql.planner.PlanNodeIdAllocator; -import com.facebook.presto.sql.planner.Symbol; -import com.facebook.presto.sql.planner.SymbolAllocator; -import com.facebook.presto.sql.planner.plan.AggregationNode; -import com.facebook.presto.sql.planner.plan.ApplyNode; -import com.facebook.presto.sql.planner.plan.PlanNode; -import com.facebook.presto.sql.planner.plan.ProjectNode; -import com.facebook.presto.sql.planner.plan.SimplePlanRewriter; -import com.facebook.presto.sql.tree.ComparisonExpression; -import com.facebook.presto.sql.tree.Expression; -import com.facebook.presto.sql.tree.FunctionCall; -import com.facebook.presto.sql.tree.LogicalBinaryExpression; -import com.facebook.presto.sql.tree.QualifiedName; -import com.facebook.presto.sql.tree.QuantifiedComparisonExpression; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; - -import java.util.EnumSet; -import java.util.List; -import java.util.Map; -import java.util.Optional; - -import static com.facebook.presto.sql.analyzer.SemanticExceptions.throwNotSupportedException; -import static com.facebook.presto.sql.analyzer.TypeSignatureProvider.fromTypeSignatures; -import static com.facebook.presto.sql.planner.plan.SimplePlanRewriter.rewriteWith; -import static com.facebook.presto.sql.tree.ComparisonExpressionType.EQUAL; -import static com.facebook.presto.sql.tree.ComparisonExpressionType.GREATER_THAN; -import static com.facebook.presto.sql.tree.ComparisonExpressionType.GREATER_THAN_OR_EQUAL; -import static com.facebook.presto.sql.tree.ComparisonExpressionType.LESS_THAN; -import static com.facebook.presto.sql.tree.ComparisonExpressionType.LESS_THAN_OR_EQUAL; -import static com.facebook.presto.sql.tree.QuantifiedComparisonExpression.Quantifier.ALL; -import static com.facebook.presto.util.ImmutableCollectors.toImmutableMap; -import static com.google.common.base.Preconditions.checkState; -import static com.google.common.collect.Iterables.getOnlyElement; -import static java.util.Objects.requireNonNull; - -public class TransformQuantifiedComparisonApplyToScalarApply - implements PlanOptimizer -{ - private final Metadata metadata; - - public TransformQuantifiedComparisonApplyToScalarApply(Metadata metadata) - { - this.metadata = requireNonNull(metadata, "metadata is null"); - } - - @Override - public PlanNode optimize(PlanNode plan, Session session, Map types, SymbolAllocator symbolAllocator, PlanNodeIdAllocator idAllocator) - { - return rewriteWith(new Rewriter(idAllocator, types, symbolAllocator, metadata), plan, null); - } - - private static class Rewriter - extends SimplePlanRewriter - - { - private final PlanNodeIdAllocator idAllocator; - private final Map types; - private final SymbolAllocator symbolAllocator; - private final Metadata metadata; - - public Rewriter(PlanNodeIdAllocator idAllocator, Map types, SymbolAllocator symbolAllocator, Metadata metadata) - { - this.idAllocator = requireNonNull(idAllocator, "idAllocator is null"); - this.types = requireNonNull(types, "types is null"); - this.symbolAllocator = requireNonNull(symbolAllocator, "symbolAllocator is null"); - this.metadata = requireNonNull(metadata, "metadata is null"); - } - - @Override - public PlanNode visitApply(ApplyNode node, RewriteContext context) - { - if (node.getSubqueryAssignments().size() != 1) { - return context.defaultRewrite(node); - } - - Expression expression = getOnlyElement(node.getSubqueryAssignments().values()); - if (!(expression instanceof QuantifiedComparisonExpression)) { - return context.defaultRewrite(node); - } - - QuantifiedComparisonExpression quantifiedComparison = (QuantifiedComparisonExpression) expression; - - if (quantifiedComparison.getComparisonType() == EQUAL && quantifiedComparison.getQuantifier() == ALL) { - // A = ALL B <=> min B = max B && A = min B - return rewriteQuantifiedEqualsAllApplyNode(node, quantifiedComparison, context); - } - - if (EnumSet.of(LESS_THAN, LESS_THAN_OR_EQUAL, GREATER_THAN, GREATER_THAN_OR_EQUAL).contains(quantifiedComparison.getComparisonType())) { - // A < ALL B <=> A < min B - // A > ALL B <=> A > max B - // A < ANY B <=> A < max B - // A > ANY B <=> A > min B - return rewriteQuantifiedOrderableApplyNode(node, quantifiedComparison, context); - } - - return context.defaultRewrite(node); - } - - private PlanNode rewriteQuantifiedEqualsAllApplyNode(ApplyNode node, QuantifiedComparisonExpression quantifiedComparison, RewriteContext context) - { - PlanNode subqueryPlan = context.rewrite(node.getSubquery()); - Symbol outputColumnSymbol = getOnlyElement(subqueryPlan.getOutputSymbols()); - Type outputColumnType = types.get(outputColumnSymbol); - if (!outputColumnType.isOrderable()) { - throwNotSupportedException(quantifiedComparison, "Quantified comparison '= ALL' or '<> ANY' for unorderable type " + outputColumnType.getDisplayName()); - } - - List outputColumnTypeSignature = ImmutableList.of(outputColumnType.getTypeSignature()); - FunctionRegistry functionRegistry = metadata.getFunctionRegistry(); - QualifiedName min = QualifiedName.of("min"); - QualifiedName max = QualifiedName.of("max"); - Symbol minValue = symbolAllocator.newSymbol(min.toString(), outputColumnType); - Symbol maxValue = symbolAllocator.newSymbol(max.toString(), outputColumnType); - List outputColumnReferences = ImmutableList.of(outputColumnSymbol.toSymbolReference()); - subqueryPlan = new AggregationNode( - idAllocator.getNextId(), - subqueryPlan, - ImmutableMap.of( - minValue, new FunctionCall(min, outputColumnReferences), - maxValue, new FunctionCall(max, outputColumnReferences) - ), - ImmutableMap.of( - minValue, functionRegistry.resolveFunction(min, fromTypeSignatures(outputColumnTypeSignature)), - maxValue, functionRegistry.resolveFunction(max, fromTypeSignatures(outputColumnTypeSignature)) - ), - ImmutableMap.of(), - ImmutableList.of(ImmutableList.of()), - AggregationNode.Step.SINGLE, - Optional.empty(), - Optional.empty()); - - PlanNode applyNode = new ApplyNode( - node.getId(), - context.rewrite(node.getInput()), - subqueryPlan, - ImmutableMap.of(minValue, minValue.toSymbolReference(), maxValue, maxValue.toSymbolReference()), - node.getCorrelation()); - - Symbol quantifiedComparisonSymbol = getOnlyElement(node.getSubqueryAssignments().keySet()); - LogicalBinaryExpression valueComparedToSubquery = new LogicalBinaryExpression(LogicalBinaryExpression.Type.AND, - new ComparisonExpression(EQUAL, minValue.toSymbolReference(), maxValue.toSymbolReference()), - new ComparisonExpression(EQUAL, quantifiedComparison.getValue(), minValue.toSymbolReference()) - ); - return projectExpressions(applyNode, ImmutableMap.of(quantifiedComparisonSymbol, valueComparedToSubquery)); - } - - private PlanNode rewriteQuantifiedOrderableApplyNode(ApplyNode node, QuantifiedComparisonExpression quantifiedComparison, RewriteContext context) - { - QualifiedName aggregationFunction = chooseAggregationFunction(quantifiedComparison); - PlanNode subqueryPlan = context.rewrite(node.getSubquery()); - Symbol outputColumn = getOnlyElement(subqueryPlan.getOutputSymbols()); - Type outputColumnType = types.get(outputColumn); - checkState(outputColumnType.isOrderable(), "Subquery result type must be orderable"); - - FunctionRegistry functionRegistry = metadata.getFunctionRegistry(); - Symbol subValue = symbolAllocator.newSymbol(aggregationFunction.toString(), outputColumnType); - subqueryPlan = new AggregationNode( - idAllocator.getNextId(), - subqueryPlan, - ImmutableMap.of(subValue, new FunctionCall(aggregationFunction, ImmutableList.of(outputColumn.toSymbolReference()))), - ImmutableMap.of(subValue, functionRegistry.resolveFunction(aggregationFunction, ImmutableList.of(new TypeSignatureProvider(outputColumnType.getTypeSignature())))), - ImmutableMap.of(), - ImmutableList.of(ImmutableList.of()), - AggregationNode.Step.SINGLE, - Optional.empty(), - Optional.empty()); - - PlanNode applyNode = new ApplyNode( - node.getId(), - context.rewrite(node.getInput()), - subqueryPlan, - ImmutableMap.of(subValue, subValue.toSymbolReference()), - node.getCorrelation()); - - ComparisonExpression valueComparedToSubquery = new ComparisonExpression(quantifiedComparison.getComparisonType(), quantifiedComparison.getValue(), subValue.toSymbolReference()); - Symbol quantifiedComparisonSymbol = getOnlyElement(node.getSubqueryAssignments().keySet()); - return projectExpressions(applyNode, ImmutableMap.of(quantifiedComparisonSymbol, valueComparedToSubquery)); - } - - private ProjectNode projectExpressions(PlanNode input, Map expressions) - { - Map identityProjections = input.getOutputSymbols().stream() - .collect(toImmutableMap(symbol -> symbol, Symbol::toSymbolReference)); - return new ProjectNode( - idAllocator.getNextId(), - input, - ImmutableMap.builder() - .putAll(identityProjections) - .putAll(expressions) - .build()); - } - - private static QualifiedName chooseAggregationFunction(QuantifiedComparisonExpression quantifiedComparison) - { - switch (quantifiedComparison.getQuantifier()) { - case ALL: - switch (quantifiedComparison.getComparisonType()) { - case LESS_THAN: - case LESS_THAN_OR_EQUAL: - return QualifiedName.of("min"); - case GREATER_THAN: - case GREATER_THAN_OR_EQUAL: - return QualifiedName.of("max"); - } - break; - case ANY: - case SOME: - switch (quantifiedComparison.getComparisonType()) { - case LESS_THAN: - case LESS_THAN_OR_EQUAL: - return QualifiedName.of("max"); - case GREATER_THAN: - case GREATER_THAN_OR_EQUAL: - return QualifiedName.of("min"); - } - break; - } - throw new IllegalArgumentException("Unexpected quantifier: " + quantifiedComparison.getQuantifier()); - } - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TransformUncorrelatedInPredicateSubqueryToSemiJoin.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TransformUncorrelatedInPredicateSubqueryToSemiJoin.java index 80d321693caf..5f2548dd92c8 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TransformUncorrelatedInPredicateSubqueryToSemiJoin.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TransformUncorrelatedInPredicateSubqueryToSemiJoin.java @@ -19,21 +19,34 @@ import com.facebook.presto.sql.planner.Symbol; import com.facebook.presto.sql.planner.SymbolAllocator; import com.facebook.presto.sql.planner.plan.ApplyNode; +import com.facebook.presto.sql.planner.plan.FilterNode; import com.facebook.presto.sql.planner.plan.PlanNode; +import com.facebook.presto.sql.planner.plan.ProjectNode; import com.facebook.presto.sql.planner.plan.SemiJoinNode; import com.facebook.presto.sql.planner.plan.SimplePlanRewriter; +import com.facebook.presto.sql.tree.DefaultTraversalVisitor; import com.facebook.presto.sql.tree.Expression; import com.facebook.presto.sql.tree.InPredicate; +import com.facebook.presto.sql.tree.SymbolReference; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; import java.util.Map; import java.util.Optional; +import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; +import static com.facebook.presto.sql.planner.ExpressionNodeInliner.replaceExpression; import static com.facebook.presto.sql.planner.plan.SimplePlanRewriter.rewriteWith; +import static com.google.common.base.Preconditions.checkState; import static com.google.common.collect.Iterables.getOnlyElement; import static java.util.Objects.requireNonNull; /** - * This optimizers looks for InPredicate expressions in ApplyNodes and replaces the nodes with SemiJoin nodes. + * This optimizers looks for InPredicate expressions with subqueries, finds matching uncorrelated Apply nodes + * and then replace Apply nodes with SemiJoin nodes and updates InPredicates. *

* Plan before optimizer: *

@@ -61,52 +74,197 @@ public class TransformUncorrelatedInPredicateSubqueryToSemiJoin
     @Override
     public PlanNode optimize(PlanNode plan, Session session, Map types, SymbolAllocator symbolAllocator, PlanNodeIdAllocator idAllocator)
     {
-        return rewriteWith(new InPredicateRewriter(idAllocator), plan, null);
+        return rewriteWith(new InPredicateRewriter(idAllocator, symbolAllocator), plan, null);
     }
 
     /**
-     * Each ApplyNode which contains InPredicate is replaced by semi join node.
+     * For each node which contains InPredicate this rewriter calls {@link InsertSemiJoinRewriter} rewriter, then
+     * InPredicate is replaced by semi join symbol returned from the used nested rewriter.
      */
     private static class InPredicateRewriter
             extends SimplePlanRewriter
     {
         private final PlanNodeIdAllocator idAllocator;
+        private final SymbolAllocator symbolAllocator;
+        private final List> inPredicateMappings = new ArrayList<>();
 
-        public InPredicateRewriter(PlanNodeIdAllocator idAllocator)
+        public InPredicateRewriter(PlanNodeIdAllocator idAllocator, SymbolAllocator symbolAllocator)
         {
             this.idAllocator = requireNonNull(idAllocator, "idAllocator is null");
+            this.symbolAllocator = requireNonNull(symbolAllocator, "symbolAllocator is null");
         }
 
         @Override
-        public PlanNode visitApply(ApplyNode node, RewriteContext context)
+        public PlanNode visitFilter(FilterNode node, RewriteContext context)
         {
-            if (!node.getCorrelation().isEmpty()) {
-                return context.defaultRewrite(node);
+            PlanNode rewrittenNode = rewriteInPredicates(
+                    context.defaultRewrite(node, context.get()),
+                    node.getPredicate());
+
+            return new FilterNode(
+                    rewrittenNode.getId(),
+                    getOnlyElement(rewrittenNode.getSources()),
+                    replaceInPredicates(node.getPredicate()));
+        }
+
+        @Override
+        public PlanNode visitProject(ProjectNode node, RewriteContext context)
+        {
+            PlanNode rewrittenNode = rewriteInPredicates(
+                    context.defaultRewrite(node, context.get()),
+                    node.getAssignments().values());
+
+            if (inPredicateMappings.isEmpty()) {
+                return rewrittenNode;
+            }
+            else {
+                return new ProjectNode(rewrittenNode.getId(),
+                        getOnlyElement(rewrittenNode.getSources()),
+                        replaceInPredicateInAssignments(node));
             }
+        }
+
+        private PlanNode rewriteInPredicates(PlanNode node, Expression expressions)
+        {
+            return rewriteInPredicates(node, ImmutableList.of(expressions));
+        }
 
-            if (node.getSubqueryAssignments().size() != 1) {
-                return context.defaultRewrite(node);
+        private PlanNode rewriteInPredicates(PlanNode node, Collection expressions)
+        {
+            List inPredicates = extractApplyInPredicates(expressions);
+            ImmutableMap.Builder inPredicateMapping = ImmutableMap.builder();
+            PlanNode rewrittenNode = node;
+            for (InPredicate inPredicate : inPredicates) {
+                InsertSemiJoinRewriter rewriter = new InsertSemiJoinRewriter(idAllocator, symbolAllocator, inPredicate);
+                rewrittenNode = rewriteWith(rewriter, rewrittenNode, null);
+                inPredicateMapping.putAll(rewriter.getInPredicateMapping());
             }
+            inPredicateMappings.add(inPredicateMapping.build());
+            return rewrittenNode;
+        }
 
-            Expression expression = getOnlyElement(node.getSubqueryAssignments().values());
-            if (!(expression instanceof InPredicate)) {
-                return context.defaultRewrite(node);
+        private Map replaceInPredicateInAssignments(ProjectNode node)
+        {
+            ImmutableMap.Builder assignmentsBuilder = ImmutableMap.builder();
+            Map assignments = node.getAssignments();
+            for (Symbol symbol : assignments.keySet()) {
+                assignmentsBuilder.put(symbol, replaceInPredicates(assignments.get(symbol)));
             }
+            return assignmentsBuilder.build();
+        }
 
-            PlanNode input = context.rewrite(node.getInput());
-            PlanNode subquery = context.rewrite(node.getSubquery());
-
-            InPredicate inPredicate = (InPredicate) expression;
-            Symbol semiJoinSymbol = getOnlyElement(node.getSubqueryAssignments().keySet());
-            return new SemiJoinNode(idAllocator.getNextId(),
-                    input,
-                    subquery,
-                    Symbol.from(inPredicate.getValue()),
-                    Symbol.from(inPredicate.getValueList()),
-                    semiJoinSymbol,
-                    Optional.empty(),
-                    Optional.empty()
-            );
+        private Expression replaceInPredicates(Expression expression)
+        {
+            for (Map inPredicateMapping : inPredicateMappings) {
+                expression = replaceExpression(expression, inPredicateMapping);
+            }
+            return expression;
+        }
+    }
+
+    private static List extractApplyInPredicates(Collection expressions)
+    {
+        ImmutableList.Builder inPredicates = ImmutableList.builder();
+        for (Expression expression : expressions) {
+            new DefaultTraversalVisitor()
+            {
+                @Override
+                protected Void visitInPredicate(InPredicate node, Void context)
+                {
+                    if (node.getValueList() instanceof SymbolReference) {
+                        inPredicates.add(node);
+                    }
+                    return null;
+                }
+            }.process(expression, null);
+        }
+        return inPredicates.build();
+    }
+
+    /**
+     * For given InPredicate (in context) it finds matching Apply node (which produces InPredicate value in apply's input,
+     * and valueList in apply's subquery) and replace it with a SemiJoin node.
+     * Between InPredicate's plan node and Apply node there could be several projections of InPredicate symbols, so they
+     * have to be considered.
+     */
+    private static class InsertSemiJoinRewriter
+            extends ApplyNodeRewriter
+    {
+        private final PlanNodeIdAllocator idAllocator;
+        private final SymbolAllocator symbolAllocator;
+        private final InPredicate originalInPredicate;
+        private InPredicate inPredicate;
+        private Optional semiJoinSymbol = Optional.empty();
+
+        public InsertSemiJoinRewriter(PlanNodeIdAllocator idAllocator, SymbolAllocator symbolAllocator, InPredicate inPredicate)
+        {
+            super((SymbolReference) inPredicate.getValueList());
+            this.idAllocator = requireNonNull(idAllocator, "idAllocator is null");
+            this.symbolAllocator = requireNonNull(symbolAllocator, "symbolAllocator is null");
+            this.inPredicate = requireNonNull(inPredicate, "inPredicate is null");
+            this.originalInPredicate = requireNonNull(inPredicate, "inPredicate is null");
+        }
+
+        @Override
+        public PlanNode visitProject(ProjectNode node, RewriteContext context)
+        {
+            inPredicate = (InPredicate) replaceExpression(
+                    inPredicate,
+                    mapAssignmentSymbolsToExpression(node.getAssignments()));
+
+            // cannot use context.rewrite() as it ends with endless loop
+            ProjectNode rewrittenNode = (ProjectNode) context.defaultRewrite(node);
+            if (semiJoinSymbol.isPresent()) {
+                return appendIdentityProjection(rewrittenNode, semiJoinSymbol.get());
+            }
+            else {
+                return rewrittenNode;
+            }
+        }
+
+        private ProjectNode appendIdentityProjection(ProjectNode node, Symbol symbol)
+        {
+            if (node.getOutputSymbols().contains(symbol)) {
+                return node;
+            }
+            else if (node.getSource().getOutputSymbols().contains(symbol)) {
+                ImmutableMap.Builder builder = ImmutableMap.builder();
+                builder.putAll(node.getAssignments());
+                builder.put(symbol, symbol.toSymbolReference());
+                return new ProjectNode(node.getId(), node.getSource(), builder.build());
+            }
+            else {
+                return node;
+            }
+        }
+
+        @Override
+        protected PlanNode rewriteApply(ApplyNode node)
+        {
+            if (node.getCorrelation().isEmpty()) {
+                Symbol value = Symbol.from(inPredicate.getValue());
+                Symbol valueList = Symbol.from(inPredicate.getValueList());
+                checkState(!semiJoinSymbol.isPresent(), "Semi join symbol is already set");
+                semiJoinSymbol = Optional.of(symbolAllocator.newSymbol("semijoin_result", BOOLEAN));
+                return new SemiJoinNode(idAllocator.getNextId(),
+                        node.getInput(),
+                        node.getSubquery(),
+                        value,
+                        valueList,
+                        semiJoinSymbol.get(),
+                        Optional.empty(),
+                        Optional.empty()
+                );
+            }
+            return node;
+        }
+
+        public Map getInPredicateMapping()
+        {
+            if (!semiJoinSymbol.isPresent()) {
+                return ImmutableMap.of();
+            }
+            return ImmutableMap.of(originalInPredicate, semiJoinSymbol.get().toSymbolReference());
         }
     }
 }
diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TransformUncorrelatedScalarToJoin.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TransformUncorrelatedScalarToJoin.java
index ae54b10b86cd..6da25ff26c49 100644
--- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TransformUncorrelatedScalarToJoin.java
+++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/TransformUncorrelatedScalarToJoin.java
@@ -19,6 +19,7 @@
 import com.facebook.presto.sql.planner.Symbol;
 import com.facebook.presto.sql.planner.SymbolAllocator;
 import com.facebook.presto.sql.planner.plan.ApplyNode;
+import com.facebook.presto.sql.planner.plan.EnforceSingleRowNode;
 import com.facebook.presto.sql.planner.plan.JoinNode;
 import com.facebook.presto.sql.planner.plan.PlanNode;
 import com.facebook.presto.sql.planner.plan.SimplePlanRewriter;
@@ -52,7 +53,7 @@ public Rewriter(PlanNodeIdAllocator idAllocator)
         public PlanNode visitApply(ApplyNode node, RewriteContext context)
         {
             ApplyNode rewrittenNode = (ApplyNode) context.defaultRewrite(node, context.get());
-            if (rewrittenNode.getCorrelation().isEmpty() && rewrittenNode.isResolvedScalarSubquery()) {
+            if (rewrittenNode.getCorrelation().isEmpty() && rewrittenNode.getSubquery() instanceof EnforceSingleRowNode) {
                 return new JoinNode(
                         idAllocator.getNextId(),
                         JoinNode.Type.INNER,
diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/UnaliasSymbolReferences.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/UnaliasSymbolReferences.java
index 696712b8f02d..153bd132298a 100644
--- a/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/UnaliasSymbolReferences.java
+++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/optimizations/UnaliasSymbolReferences.java
@@ -82,10 +82,8 @@
 import java.util.Optional;
 import java.util.Set;
 
-import static com.facebook.presto.sql.planner.plan.JoinNode.Type.INNER;
 import static com.facebook.presto.util.ImmutableCollectors.toImmutableList;
 import static com.facebook.presto.util.ImmutableCollectors.toImmutableSet;
-import static com.google.common.base.Preconditions.checkState;
 import static java.util.Objects.requireNonNull;
 
 /**
@@ -111,19 +109,13 @@ public PlanNode optimize(PlanNode plan, Session session, Map types
         requireNonNull(symbolAllocator, "symbolAllocator is null");
         requireNonNull(idAllocator, "idAllocator is null");
 
-        return SimplePlanRewriter.rewriteWith(new Rewriter(types), plan);
+        return SimplePlanRewriter.rewriteWith(new Rewriter(), plan);
     }
 
     private static class Rewriter
             extends SimplePlanRewriter
     {
         private final Map mapping = new HashMap<>();
-        private final Map types;
-
-        private Rewriter(Map types)
-        {
-            this.types = types;
-        }
 
         @Override
         public PlanNode visitAggregation(AggregationNode node, RewriteContext context)
@@ -280,8 +272,6 @@ public PlanNode visitExchange(ExchangeNode node, RewriteContext context)
                     .map(context::rewrite)
                     .collect(toImmutableList());
 
-            mapExchangeNodeSymbols(node);
-
             List> inputs = new ArrayList<>();
             for (int i = 0; i < node.getInputs().size(); i++) {
                 inputs.add(new ArrayList<>());
@@ -309,52 +299,6 @@ public PlanNode visitExchange(ExchangeNode node, RewriteContext context)
             return new ExchangeNode(node.getId(), node.getType(), node.getScope(), partitioningScheme, sources, inputs);
         }
 
-        private void mapExchangeNodeSymbols(ExchangeNode node)
-        {
-            if (node.getInputs().size() == 1) {
-                mapExchangeNodeOutputToInputSymbols(node);
-                return;
-            }
-
-            // Mapping from list [node.getInput(0).get(symbolIndex), node.getInput(1).get(symbolIndex), ...] to node.getOutputSymbols(symbolIndex).
-            // All symbols are canonical.
-            Map, Symbol> inputsToOutputs = new HashMap<>();
-            // Map each same list of input symbols [I1, I2, ..., In] to the same output symbol O
-            for (int symbolIndex = 0; symbolIndex < node.getOutputSymbols().size(); symbolIndex++) {
-                Symbol canonicalOutput = canonicalize(node.getOutputSymbols().get(symbolIndex));
-                List canonicalInputs = canonicalizeExchangeNodeInputs(node, symbolIndex);
-                Symbol output = inputsToOutputs.get(canonicalInputs);
-
-                if (output == null || canonicalOutput.equals(output)) {
-                    inputsToOutputs.put(canonicalInputs, canonicalOutput);
-                }
-                else {
-                    map(canonicalOutput, output);
-                }
-            }
-        }
-
-        private void mapExchangeNodeOutputToInputSymbols(ExchangeNode node)
-        {
-            checkState(node.getInputs().size() == 1);
-
-            for (int symbolIndex = 0; symbolIndex < node.getOutputSymbols().size(); symbolIndex++) {
-                Symbol canonicalOutput = canonicalize(node.getOutputSymbols().get(symbolIndex));
-                Symbol canonicalInput = canonicalize(node.getInputs().get(0).get(symbolIndex));
-
-                if (!canonicalOutput.equals(canonicalInput)) {
-                    map(canonicalOutput, canonicalInput);
-                }
-            }
-        }
-
-        private List canonicalizeExchangeNodeInputs(ExchangeNode node, int symbolIndex)
-        {
-            return node.getInputs().stream()
-                    .map(input -> canonicalize(input.get(symbolIndex)))
-                    .collect(toImmutableList());
-        }
-
         @Override
         public PlanNode visitRemoteSource(RemoteSourceNode node, RewriteContext context)
         {
@@ -428,7 +372,43 @@ public PlanNode visitFilter(FilterNode node, RewriteContext context)
         public PlanNode visitProject(ProjectNode node, RewriteContext context)
         {
             PlanNode source = context.rewrite(node.getSource());
-            return new ProjectNode(node.getId(), source, canonicalize(node.getAssignments()));
+
+            Map computedExpressions = new HashMap<>();
+
+            Map assignments = new LinkedHashMap<>();
+            for (Map.Entry entry : node.getAssignments().entrySet()) {
+                Expression expression = canonicalize(entry.getValue());
+
+                if (expression instanceof SymbolReference) {
+                    // Always map a trivial symbol projection
+                    Symbol symbol = Symbol.from(expression);
+                    if (!symbol.equals(entry.getKey())) {
+                        map(entry.getKey(), symbol);
+                    }
+                }
+                else if (DeterminismEvaluator.isDeterministic(expression) && !(expression instanceof NullLiteral)) {
+                    // Try to map same deterministic expressions within a projection into the same symbol
+                    // Omit NullLiterals since those have ambiguous types
+                    Symbol computedSymbol = computedExpressions.get(expression);
+                    if (computedSymbol == null) {
+                        // If we haven't seen the expression before in this projection, record it
+                        computedExpressions.put(expression, entry.getKey());
+                    }
+                    else {
+                        // If we have seen the expression before and if it is deterministic
+                        // then we can rewrite references to the current symbol in terms of the parallel computedSymbol in the projection
+                        map(entry.getKey(), computedSymbol);
+                    }
+                }
+
+                Symbol canonical = canonicalize(entry.getKey());
+
+                if (!assignments.containsKey(canonical)) {
+                    assignments.put(canonical, expression);
+                }
+            }
+
+            return new ProjectNode(node.getId(), source, assignments);
         }
 
         @Override
@@ -463,7 +443,7 @@ public PlanNode visitApply(ApplyNode node, RewriteContext context)
             PlanNode subquery = context.rewrite(node.getSubquery());
             List canonicalCorrelation = Lists.transform(node.getCorrelation(), this::canonicalize);
 
-            return new ApplyNode(node.getId(), source, subquery, canonicalize(node.getSubqueryAssignments()), canonicalCorrelation);
+            return new ApplyNode(node.getId(), source, subquery, canonicalCorrelation);
         }
 
         @Override
@@ -504,18 +484,7 @@ public PlanNode visitJoin(JoinNode node, RewriteContext context)
             PlanNode left = context.rewrite(node.getLeft());
             PlanNode right = context.rewrite(node.getRight());
 
-            List canonicalCriteria = canonicalizeJoinCriteria(node.getCriteria());
-            Optional canonicalFilter = node.getFilter().map(this::canonicalize);
-            Optional canonicalLeftHashSymbol = canonicalize(node.getLeftHashSymbol());
-            Optional canonicalRightHashSymbol = canonicalize(node.getRightHashSymbol());
-
-            if (node.getType().equals(INNER)) {
-                canonicalCriteria.stream()
-                        .filter(clause -> types.get(clause.getLeft()).equals(types.get(clause.getRight())))
-                        .forEach(clause -> map(clause.getRight(), clause.getLeft()));
-            }
-
-            return new JoinNode(node.getId(), node.getType(), left, right, canonicalCriteria, canonicalFilter, canonicalLeftHashSymbol, canonicalRightHashSymbol);
+            return new JoinNode(node.getId(), node.getType(), left, right, canonicalizeJoinCriteria(node.getCriteria()), node.getFilter().map(this::canonicalize), canonicalize(node.getLeftHashSymbol()), canonicalize(node.getRightHashSymbol()));
         }
 
         @Override
@@ -545,22 +514,22 @@ public PlanNode visitIndexJoin(IndexJoinNode node, RewriteContext context)
         @Override
         public PlanNode visitUnion(UnionNode node, RewriteContext context)
         {
-            return new UnionNode(node.getId(), rewriteSources(node, context).build(), canonicalizeSetOperationSymbolMap(node.getSymbolMapping()), canonicalizeAndDistinct(node.getOutputSymbols()));
+            return new UnionNode(node.getId(), rewriteSources(node, context).build(), canonicalizeSetOperationSymbolMap(node.getSymbolMapping()), canonicalize(node.getOutputSymbols()));
         }
 
         @Override
         public PlanNode visitIntersect(IntersectNode node, RewriteContext context)
         {
-            return new IntersectNode(node.getId(), rewriteSources(node, context).build(), canonicalizeSetOperationSymbolMap(node.getSymbolMapping()), canonicalizeAndDistinct(node.getOutputSymbols()));
+            return new IntersectNode(node.getId(), rewriteSources(node, context).build(), canonicalizeSetOperationSymbolMap(node.getSymbolMapping()), canonicalize(node.getOutputSymbols()));
         }
 
         @Override
         public PlanNode visitExcept(ExceptNode node, RewriteContext context)
         {
-            return new ExceptNode(node.getId(), rewriteSources(node, context).build(), canonicalizeSetOperationSymbolMap(node.getSymbolMapping()), canonicalizeAndDistinct(node.getOutputSymbols()));
+            return new ExceptNode(node.getId(), rewriteSources(node, context).build(), canonicalizeSetOperationSymbolMap(node.getSymbolMapping()), canonicalize(node.getOutputSymbols()));
         }
 
-        private static ImmutableList.Builder rewriteSources(SetOperationNode node, RewriteContext context)
+        private ImmutableList.Builder rewriteSources(SetOperationNode node, RewriteContext context)
         {
             ImmutableList.Builder rewrittenSources = ImmutableList.builder();
             for (PlanNode source : node.getSources()) {
@@ -601,44 +570,6 @@ private void map(Symbol symbol, Symbol canonical)
             mapping.put(symbol, canonical);
         }
 
-        private Map canonicalize(Map oldAssignments)
-        {
-            Map computedExpressions = new HashMap<>();
-            Map assignments = new LinkedHashMap<>();
-            for (Map.Entry entry : oldAssignments.entrySet()) {
-                Expression expression = canonicalize(entry.getValue());
-
-                if (expression instanceof SymbolReference) {
-                    // Always map a trivial symbol projection
-                    Symbol symbol = Symbol.from(expression);
-                    if (!symbol.equals(entry.getKey())) {
-                        map(entry.getKey(), symbol);
-                    }
-                }
-                else if (DeterminismEvaluator.isDeterministic(expression) && !(expression instanceof NullLiteral)) {
-                    // Try to map same deterministic expressions within a projection into the same symbol
-                    // Omit NullLiterals since those have ambiguous types
-                    Symbol computedSymbol = computedExpressions.get(expression);
-                    if (computedSymbol == null) {
-                        // If we haven't seen the expression before in this projection, record it
-                        computedExpressions.put(expression, entry.getKey());
-                    }
-                    else {
-                        // If we have seen the expression before and if it is deterministic
-                        // then we can rewrite references to the current symbol in terms of the parallel computedSymbol in the projection
-                        map(entry.getKey(), computedSymbol);
-                    }
-                }
-
-                Symbol canonical = canonicalize(entry.getKey());
-
-                if (!assignments.containsKey(canonical)) {
-                    assignments.put(canonical, expression);
-                }
-            }
-            return assignments;
-        }
-
         private Optional canonicalize(Optional symbol)
         {
             if (symbol.isPresent()) {
@@ -695,6 +626,13 @@ private WindowNode.Specification canonicalizeAndDistinct(WindowNode.Specificatio
                     orderings.build());
         }
 
+        private List canonicalize(List symbols)
+        {
+            return symbols.stream()
+                    .map(this::canonicalize)
+                    .collect(toImmutableList());
+        }
+
         private Set canonicalize(Set symbols)
         {
             return symbols.stream()
@@ -725,12 +663,8 @@ private List canonicalizeIndexJoinCriteria(List canonicalizeSetOperationSymbolMap(ListMultimap setOperationSymbolMap)
         {
             ImmutableListMultimap.Builder builder = ImmutableListMultimap.builder();
-            Set addedSymbols = new HashSet<>();
             for (Map.Entry> entry : setOperationSymbolMap.asMap().entrySet()) {
-                Symbol canonicalOutputSymbol = canonicalize(entry.getKey());
-                if (addedSymbols.add(canonicalOutputSymbol)) {
-                    builder.putAll(canonicalOutputSymbol, Iterables.transform(entry.getValue(), this::canonicalize));
-                }
+                builder.putAll(canonicalize(entry.getKey()), Iterables.transform(entry.getValue(), this::canonicalize));
             }
             return builder.build();
         }
diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/AggregationNode.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/AggregationNode.java
index 8caabb3f70c7..5bc23ae265f2 100644
--- a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/AggregationNode.java
+++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/AggregationNode.java
@@ -72,26 +72,6 @@ public boolean isOutputPartial()
         {
             return outputPartial;
         }
-
-        public static Step partialOutput(Step step)
-        {
-            if (step.isInputRaw()) {
-                return Step.PARTIAL;
-            }
-            else {
-                return Step.INTERMEDIATE;
-            }
-        }
-
-        public static Step partialInput(Step step)
-        {
-            if (step.isOutputPartial()) {
-                return Step.INTERMEDIATE;
-            }
-            else {
-                return Step.FINAL;
-            }
-        }
     }
 
     @JsonCreator
diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/ApplyNode.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/ApplyNode.java
index f195d6fe94c1..669c108af734 100644
--- a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/ApplyNode.java
+++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/ApplyNode.java
@@ -14,8 +14,6 @@
 package com.facebook.presto.sql.planner.plan;
 
 import com.facebook.presto.sql.planner.Symbol;
-import com.facebook.presto.sql.tree.Expression;
-import com.facebook.presto.sql.tree.SymbolReference;
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.collect.ImmutableList;
@@ -23,9 +21,7 @@
 import javax.annotation.concurrent.Immutable;
 
 import java.util.List;
-import java.util.Map;
 
-import static com.facebook.presto.sql.planner.optimizations.ScalarQueryUtil.isScalar;
 import static com.google.common.base.Preconditions.checkArgument;
 import static java.util.Objects.requireNonNull;
 
@@ -41,61 +37,25 @@ public class ApplyNode
      */
     private final List correlation;
 
-    /**
-     * Expressions that use subquery symbols.
-     * 

- * Subquery expressions are different than other expressions - * in a sense that they might use an entire subquery result - * as an input (e.g: "x IN (subquery)", "x < ALL (subquery)"). - * Such expressions are invalid in linear operator context - * (e.g: ProjectNode) in logical plan, but are correct in - * ApplyNode context. - *

- * Example 1: - * - expression: input_symbol_X IN (subquery_symbol_Y) - * - meaning: if set consisting of all values for subquery_symbol_Y contains value represented by input_symbol_X - *

- * Example 2: - * - expression: input_symbol_X < ALL (subquery_symbol_Y) - * - meaning: if input_symbol_X is smaller than all subquery values represented by subquery_symbol_Y - *

- * Example 3: - * - expression: subquery_symbol_Y - * - meaning: subquery is scalar (might be enforced), therefore subquery_symbol_Y can be used directly in the rest of the plan - */ - private final Map subqueryAssignments; - @JsonCreator public ApplyNode( @JsonProperty("id") PlanNodeId id, @JsonProperty("input") PlanNode input, @JsonProperty("subquery") PlanNode subquery, - @JsonProperty("subqueryAssignments") Map subqueryAssignments, @JsonProperty("correlation") List correlation) { super(id); requireNonNull(input, "input is null"); requireNonNull(subquery, "right is null"); - requireNonNull(subqueryAssignments, "assignments is null"); requireNonNull(correlation, "correlation is null"); checkArgument(input.getOutputSymbols().containsAll(correlation), "Input does not contain symbols from correlation"); this.input = input; this.subquery = subquery; - this.subqueryAssignments = subqueryAssignments; this.correlation = ImmutableList.copyOf(correlation); } - /** - * @return true when subquery is scalar and it's output symbols are directly mapped to ApplyNode output symbols - */ - public boolean isResolvedScalarSubquery() - { - return isScalar(subquery) && subqueryAssignments.values().stream() - .allMatch(expression -> expression instanceof SymbolReference); - } - @JsonProperty("input") public PlanNode getInput() { @@ -108,12 +68,6 @@ public PlanNode getSubquery() return subquery; } - @JsonProperty("subqueryAssignments") - public Map getSubqueryAssignments() - { - return subqueryAssignments; - } - @JsonProperty("correlation") public List getCorrelation() { @@ -132,7 +86,7 @@ public List getOutputSymbols() { return ImmutableList.builder() .addAll(input.getOutputSymbols()) - .addAll(subqueryAssignments.keySet()) + .addAll(subquery.getOutputSymbols()) .build(); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/ChildReplacer.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/ChildReplacer.java index 5d71fae75c1e..a760186471c2 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/ChildReplacer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/ChildReplacer.java @@ -268,7 +268,7 @@ public PlanNode visitEnforceSingleRow(EnforceSingleRowNode node, List public PlanNode visitApply(ApplyNode node, List newChildren) { checkArgument(newChildren.size() == 2, "expected newChildren to contain 2 nodes"); - return new ApplyNode(node.getId(), newChildren.get(0), newChildren.get(1), node.getSubqueryAssignments(), node.getCorrelation()); + return new ApplyNode(node.getId(), newChildren.get(0), newChildren.get(1), node.getCorrelation()); } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/ExchangeNode.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/ExchangeNode.java index a70b7e4cd597..71926fecbd21 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/ExchangeNode.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/ExchangeNode.java @@ -79,7 +79,6 @@ public ExchangeNode( requireNonNull(partitioningScheme, "partitioningScheme is null"); requireNonNull(inputs, "inputs is null"); - checkArgument(!inputs.isEmpty(), "inputs is empty"); checkArgument(inputs.stream().allMatch(inputSymbols -> inputSymbols.size() == partitioningScheme.getOutputLayout().size()), "Input symbols do not match output symbols"); checkArgument(inputs.size() == sources.size(), "Must have same number of input lists as sources"); for (int i = 0; i < inputs.size(); i++) { diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/JoinNode.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/JoinNode.java index 6fac37d83ba3..12352ffaa061 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/JoinNode.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/plan/JoinNode.java @@ -23,7 +23,6 @@ import javax.annotation.concurrent.Immutable; import java.util.List; -import java.util.Objects; import java.util.Optional; import static java.util.Objects.requireNonNull; @@ -194,28 +193,5 @@ public Symbol getRight() { return right; } - - @Override - public boolean equals(Object obj) - { - if (this == obj) { - return true; - } - - if (obj == null || !this.getClass().equals(obj.getClass())) { - return false; - } - - EquiJoinClause other = (EquiJoinClause) obj; - - return Objects.equals(this.left, other.left) && - Objects.equals(this.right, other.right); - } - - @Override - public int hashCode() - { - return Objects.hash(left, right); - } } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/PlanSanityChecker.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/PlanSanityChecker.java index 00865fe00479..2db92a4595fa 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/PlanSanityChecker.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/PlanSanityChecker.java @@ -34,8 +34,7 @@ public final class PlanSanityChecker new TypeValidator(), new NoSubqueryExpressionLeftChecker(), new NoApplyNodeLeftChecker(), - new VerifyNoFilteredAggregations(), - new VerifyOnlyOneOutputNode()); + new VerifyNoFilteredAggregations()); private PlanSanityChecker() {} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/ValidateDependenciesChecker.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/ValidateDependenciesChecker.java index de2edd8c1e03..847d91e3de61 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/ValidateDependenciesChecker.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/ValidateDependenciesChecker.java @@ -583,16 +583,6 @@ public Void visitApply(ApplyNode node, Set boundSymbols) checkDependencies(node.getInput().getOutputSymbols(), node.getCorrelation(), "APPLY input must provide all the necessary correlation symbols for subquery"); checkDependencies(DependencyExtractor.extractUnique(node.getSubquery()), node.getCorrelation(), "not all APPLY correlation symbols are used in subquery"); - ImmutableSet inputs = ImmutableSet.builder() - .addAll(createInputs(node.getSubquery(), boundSymbols)) - .addAll(createInputs(node.getInput(), boundSymbols)) - .build(); - - for (Expression expression : node.getSubqueryAssignments().values()) { - Set dependencies = DependencyExtractor.extractUnique(expression); - checkDependencies(inputs, dependencies, "Invalid node. Expression dependencies (%s) not in source plan output (%s)", dependencies, inputs); - } - verifyUniqueId(node); return null; @@ -606,7 +596,7 @@ private void verifyUniqueId(PlanNode node) nodesById.put(id, node); } - private static ImmutableSet createInputs(PlanNode source, Set boundSymbols) + private ImmutableSet createInputs(PlanNode source, Set boundSymbols) { return ImmutableSet.builder() .addAll(source.getOutputSymbols()) diff --git a/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/VerifyOnlyOneOutputNode.java b/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/VerifyOnlyOneOutputNode.java deleted file mode 100644 index 8086b5e329cc..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/sql/planner/sanity/VerifyOnlyOneOutputNode.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.sql.planner.sanity; - -import com.facebook.presto.Session; -import com.facebook.presto.metadata.Metadata; -import com.facebook.presto.spi.type.Type; -import com.facebook.presto.sql.parser.SqlParser; -import com.facebook.presto.sql.planner.SimplePlanVisitor; -import com.facebook.presto.sql.planner.Symbol; -import com.facebook.presto.sql.planner.plan.OutputNode; -import com.facebook.presto.sql.planner.plan.PlanNode; - -import java.util.Map; - -import static com.google.common.base.Preconditions.checkState; - -public final class VerifyOnlyOneOutputNode - implements PlanSanityChecker.Checker -{ - @Override - public void validate(PlanNode plan, Session session, Metadata metadata, SqlParser sqlParser, Map types) - { - plan.accept(new Visitor(), false); - } - - private static class Visitor - extends SimplePlanVisitor - { - @Override - public Void visitOutput(OutputNode node, Boolean hasOutputNode) - { - checkState(!hasOutputNode, "Plan has multiple instances of OutputNode"); - return super.visitOutput(node, true); - } - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/relational/DeterminismEvaluator.java b/presto-main/src/main/java/com/facebook/presto/sql/relational/DeterminismEvaluator.java index 203720d6b537..91fe3e8d5bf2 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/relational/DeterminismEvaluator.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/relational/DeterminismEvaluator.java @@ -65,17 +65,5 @@ public Boolean visitCall(CallExpression call, Void context) return call.getArguments().stream() .allMatch(expression -> expression.accept(this, context)); } - - @Override - public Boolean visitLambda(LambdaDefinitionExpression lambda, Void context) - { - return lambda.getBody().accept(this, context); - } - - @Override - public Boolean visitVariableReference(VariableReferenceExpression reference, Void context) - { - return true; - } } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/relational/Expressions.java b/presto-main/src/main/java/com/facebook/presto/sql/relational/Expressions.java index 81283e4ee46d..84ae0736bc2f 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/relational/Expressions.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/relational/Expressions.java @@ -81,21 +81,6 @@ public Void visitConstant(ConstantExpression literal, Void context) builder.add(literal); return null; } - - @Override - public Void visitLambda(LambdaDefinitionExpression lambda, Void context) - { - builder.add(lambda); - lambda.getBody().accept(this, context); - return null; - } - - @Override - public Void visitVariableReference(VariableReferenceExpression reference, Void context) - { - builder.add(reference); - return null; - } }, null); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/relational/LambdaDefinitionExpression.java b/presto-main/src/main/java/com/facebook/presto/sql/relational/LambdaDefinitionExpression.java deleted file mode 100644 index 4d0246660f5a..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/sql/relational/LambdaDefinitionExpression.java +++ /dev/null @@ -1,95 +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.sql.relational; - -import com.facebook.presto.spi.type.Type; -import com.facebook.presto.type.FunctionType; -import com.google.common.base.Joiner; -import com.google.common.collect.ImmutableList; - -import java.util.List; -import java.util.Objects; - -import static com.google.common.base.Preconditions.checkArgument; -import static java.util.Objects.requireNonNull; - -public final class LambdaDefinitionExpression - extends RowExpression -{ - private final List argumentTypes; - private final List arguments; - private final RowExpression body; - - public LambdaDefinitionExpression(List argumentTypes, List arguments, RowExpression body) - { - this.argumentTypes = ImmutableList.copyOf(requireNonNull(argumentTypes, "argumentTypes is null")); - this.arguments = ImmutableList.copyOf(requireNonNull(arguments, "arguments is null")); - checkArgument(argumentTypes.size() == arguments.size(), "Number of argument types does not match number of arguments"); - this.body = requireNonNull(body, "body is null"); - } - - public List getArgumentTypes() - { - return argumentTypes; - } - - public List getArguments() - { - return arguments; - } - - public RowExpression getBody() - { - return body; - } - - @Override - public Type getType() - { - return new FunctionType(argumentTypes, body.getType()); - } - - @Override - public String toString() - { - return "(" + Joiner.on("").join(arguments) + ") -> " + body; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - LambdaDefinitionExpression that = (LambdaDefinitionExpression) o; - return Objects.equals(argumentTypes, that.argumentTypes) && - Objects.equals(arguments, that.arguments) && - Objects.equals(body, that.body); - } - - @Override - public int hashCode() - { - return Objects.hash(argumentTypes, arguments, body); - } - - @Override - public R accept(RowExpressionVisitor visitor, C context) - { - return visitor.visitLambda(this, context); - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/relational/RowExpressionVisitor.java b/presto-main/src/main/java/com/facebook/presto/sql/relational/RowExpressionVisitor.java index 91edb5577e37..e4fde2461785 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/relational/RowExpressionVisitor.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/relational/RowExpressionVisitor.java @@ -18,6 +18,4 @@ public interface RowExpressionVisitor R visitCall(CallExpression call, C context); R visitInputReference(InputReferenceExpression reference, C context); R visitConstant(ConstantExpression literal, C context); - R visitLambda(LambdaDefinitionExpression lambda, C context); - R visitVariableReference(VariableReferenceExpression reference, C context); } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlToRowExpressionTranslator.java b/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlToRowExpressionTranslator.java index fdb406eb47ae..096c4cd872f3 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlToRowExpressionTranslator.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/relational/SqlToRowExpressionTranslator.java @@ -48,8 +48,6 @@ import com.facebook.presto.sql.tree.IntervalLiteral; import com.facebook.presto.sql.tree.IsNotNullPredicate; import com.facebook.presto.sql.tree.IsNullPredicate; -import com.facebook.presto.sql.tree.LambdaArgumentDeclaration; -import com.facebook.presto.sql.tree.LambdaExpression; import com.facebook.presto.sql.tree.LikePredicate; import com.facebook.presto.sql.tree.LogicalBinaryExpression; import com.facebook.presto.sql.tree.LongLiteral; @@ -61,7 +59,6 @@ import com.facebook.presto.sql.tree.SimpleCaseExpression; import com.facebook.presto.sql.tree.StringLiteral; import com.facebook.presto.sql.tree.SubscriptExpression; -import com.facebook.presto.sql.tree.SymbolReference; import com.facebook.presto.sql.tree.TimeLiteral; import com.facebook.presto.sql.tree.TimestampLiteral; import com.facebook.presto.sql.tree.TryExpression; @@ -321,27 +318,6 @@ protected RowExpression visitFunctionCall(FunctionCall node, Void context) return call(signature, types.get(node), arguments); } - @Override - protected RowExpression visitSymbolReference(SymbolReference node, Void context) - { - return new VariableReferenceExpression(node.getName(), types.get(node)); - } - - @Override - protected RowExpression visitLambdaExpression(LambdaExpression node, Void context) - { - RowExpression body = process(node.getBody(), context); - - Type type = types.get(node); - List typeParameters = type.getTypeParameters(); - List argumentTypes = typeParameters.subList(0, typeParameters.size() - 1); - List argumentNames = node.getArguments().stream() - .map(LambdaArgumentDeclaration::getName) - .collect(toImmutableList()); - - return new LambdaDefinitionExpression(argumentTypes, argumentNames, body); - } - @Override protected RowExpression visitArithmeticBinary(ArithmeticBinaryExpression node, Void context) { @@ -399,7 +375,7 @@ protected RowExpression visitCast(Cast node, Void context) return call(castSignature(types.get(node), value.getType()), types.get(node), value); } - private static RowExpression changeType(RowExpression value, Type targetType) + private RowExpression changeType(RowExpression value, Type targetType) { ChangeTypeVisitor visitor = new ChangeTypeVisitor(targetType); return value.accept(visitor, null); @@ -432,18 +408,6 @@ public RowExpression visitConstant(ConstantExpression literal, Void context) { return new ConstantExpression(literal.getValue(), targetType); } - - @Override - public RowExpression visitLambda(LambdaDefinitionExpression lambda, Void context) - { - throw new UnsupportedOperationException(); - } - - @Override - public RowExpression visitVariableReference(VariableReferenceExpression reference, Void context) - { - return new VariableReferenceExpression(reference.getName(), targetType); - } } @Override diff --git a/presto-main/src/main/java/com/facebook/presto/sql/relational/VariableReferenceExpression.java b/presto-main/src/main/java/com/facebook/presto/sql/relational/VariableReferenceExpression.java deleted file mode 100644 index 51787902b3e7..000000000000 --- a/presto-main/src/main/java/com/facebook/presto/sql/relational/VariableReferenceExpression.java +++ /dev/null @@ -1,75 +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.sql.relational; - -import com.facebook.presto.spi.type.Type; - -import java.util.Objects; - -import static java.util.Objects.requireNonNull; - -public final class VariableReferenceExpression - extends RowExpression -{ - private final String name; - private final Type type; - - public VariableReferenceExpression(String name, Type type) - { - this.name = requireNonNull(name, "name is null"); - this.type = requireNonNull(type, "type is null"); - } - - public String getName() - { - return name; - } - - @Override - public Type getType() - { - return type; - } - - @Override - public int hashCode() - { - return Objects.hash(name, type); - } - - @Override - public String toString() - { - return name; - } - - @Override - public R accept(RowExpressionVisitor visitor, C context) - { - return visitor.visitVariableReference(this, context); - } - - @Override - public boolean equals(Object obj) - { - if (this == obj) { - return true; - } - if (obj == null || getClass() != obj.getClass()) { - return false; - } - VariableReferenceExpression other = (VariableReferenceExpression) obj; - return Objects.equals(this.name, other.name) && Objects.equals(this.type, other.type); - } -} diff --git a/presto-main/src/main/java/com/facebook/presto/sql/relational/optimizer/ExpressionOptimizer.java b/presto-main/src/main/java/com/facebook/presto/sql/relational/optimizer/ExpressionOptimizer.java index be8f4b1322f8..7dd9f3894802 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/relational/optimizer/ExpressionOptimizer.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/relational/optimizer/ExpressionOptimizer.java @@ -22,10 +22,8 @@ import com.facebook.presto.sql.relational.CallExpression; import com.facebook.presto.sql.relational.ConstantExpression; import com.facebook.presto.sql.relational.InputReferenceExpression; -import com.facebook.presto.sql.relational.LambdaDefinitionExpression; import com.facebook.presto.sql.relational.RowExpression; import com.facebook.presto.sql.relational.RowExpressionVisitor; -import com.facebook.presto.sql.relational.VariableReferenceExpression; import com.google.common.collect.Iterables; import java.lang.invoke.MethodHandle; @@ -151,7 +149,6 @@ public RowExpression visitCall(CallExpression call, Void context) .map(argument -> argument.accept(this, context)) .collect(toImmutableList()); - // TODO: optimize function calls with lambda arguments. For example, apply(x -> x + 2, 1) if (Iterables.all(arguments, instanceOf(ConstantExpression.class)) && function.isDeterministic()) { MethodHandle method = function.getMethodHandle(); @@ -184,17 +181,5 @@ public RowExpression visitCall(CallExpression call, Void context) return call(signature, typeManager.getType(signature.getReturnType()), arguments); } - - @Override - public RowExpression visitLambda(LambdaDefinitionExpression lambda, Void context) - { - return new LambdaDefinitionExpression(lambda.getArgumentTypes(), lambda.getArguments(), lambda.getBody().accept(this, context)); - } - - @Override - public RowExpression visitVariableReference(VariableReferenceExpression reference, Void context) - { - return reference; - } } } diff --git a/presto-main/src/main/java/com/facebook/presto/sql/rewrite/DescribeInputRewrite.java b/presto-main/src/main/java/com/facebook/presto/sql/rewrite/DescribeInputRewrite.java index 9bfcdb9bd1e3..e96020adedbb 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/rewrite/DescribeInputRewrite.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/rewrite/DescribeInputRewrite.java @@ -127,7 +127,7 @@ protected Node visitDescribeInput(DescribeInput node, Void context) ); } - private static Row createDescribeInputRow(Parameter parameter, Analysis queryAnalysis) + private Row createDescribeInputRow(Parameter parameter, Analysis queryAnalysis) { Type type = queryAnalysis.getCoercion(parameter); if (type == null) { diff --git a/presto-main/src/main/java/com/facebook/presto/sql/rewrite/DescribeOutputRewrite.java b/presto-main/src/main/java/com/facebook/presto/sql/rewrite/DescribeOutputRewrite.java index 44d1415ec760..d8fab4c5d79b 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/rewrite/DescribeOutputRewrite.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/rewrite/DescribeOutputRewrite.java @@ -125,7 +125,7 @@ protected Node visitDescribeOutput(DescribeOutput node, Void context) limit); } - private static Row createDescribeOutputRow(Field field, Analysis analysis) + private Row createDescribeOutputRow(Field field, Analysis analysis) { LongLiteral typeSize = new LongLiteral("0"); if (field.getType() instanceof FixedWidthType) { diff --git a/presto-main/src/main/java/com/facebook/presto/sql/rewrite/ShowQueriesRewrite.java b/presto-main/src/main/java/com/facebook/presto/sql/rewrite/ShowQueriesRewrite.java index ae668481966c..90062331d157 100644 --- a/presto-main/src/main/java/com/facebook/presto/sql/rewrite/ShowQueriesRewrite.java +++ b/presto-main/src/main/java/com/facebook/presto/sql/rewrite/ShowQueriesRewrite.java @@ -244,7 +244,6 @@ protected Node visitShowColumns(ShowColumns showColumns, Void context) selectList( aliasedName("column_name", "Column"), aliasedName("data_type", "Type"), - aliasedNullToEmpty("extra_info", "Extra"), aliasedNullToEmpty("comment", "Comment")), from(tableName.getCatalogName(), TABLE_COLUMNS), logicalAnd( diff --git a/presto-main/src/main/java/com/facebook/presto/testing/LocalQueryRunner.java b/presto-main/src/main/java/com/facebook/presto/testing/LocalQueryRunner.java index 42d7f6e77ae9..58946004ff09 100644 --- a/presto-main/src/main/java/com/facebook/presto/testing/LocalQueryRunner.java +++ b/presto-main/src/main/java/com/facebook/presto/testing/LocalQueryRunner.java @@ -203,6 +203,7 @@ public class LocalQueryRunner private final PageIndexerFactory pageIndexerFactory; private final MetadataManager metadata; private final TestingAccessControlManager accessControl; + private final TestingEventListenerManager eventListener; private final SplitManager splitManager; private final BlockEncodingSerde blockEncodingSerde; private final PageSourceManager pageSourceManager; @@ -272,6 +273,7 @@ private LocalQueryRunner(Session defaultSession, FeaturesConfig featuresConfig, new TablePropertyManager(), transactionManager); this.accessControl = new TestingAccessControlManager(transactionManager); + this.eventListener = new TestingEventListenerManager(); this.pageSourceManager = new PageSourceManager(); this.expressionCompiler = new ExpressionCompiler(metadata); @@ -330,7 +332,7 @@ private LocalQueryRunner(Session defaultSession, FeaturesConfig featuresConfig, dataDefinitionTask = ImmutableMap., DataDefinitionTask>builder() .put(CreateTable.class, new CreateTableTask()) - .put(CreateView.class, new CreateViewTask(jsonCodec(ViewDefinition.class), sqlParser, new FeaturesConfig())) + .put(CreateView.class, new CreateViewTask(jsonCodec(ViewDefinition.class), sqlParser, accessControl, new FeaturesConfig())) .put(DropTable.class, new DropTableTask()) .put(DropView.class, new DropViewTask()) .put(RenameColumn.class, new RenameColumnTask()) @@ -540,7 +542,7 @@ public List createDrivers(Session session, @Language("SQL") String sql, System.out.println(PlanPrinter.textLogicalPlan(plan.getRoot(), plan.getTypes(), metadata, session)); } - SubPlan subplan = PlanFragmenter.createSubPlans(session, metadata, plan); + SubPlan subplan = new PlanFragmenter().createSubPlans(session, metadata, plan); if (!subplan.getChildren().isEmpty()) { throw new AssertionError("Expected subplan to have no children"); } @@ -637,15 +639,15 @@ public Plan createPlan(Session session, @Language("SQL") String sql, LogicalPlan .setDistributedIndexJoinsEnabled(false) .setOptimizeHashGeneration(true); PlanOptimizers planOptimizers = new PlanOptimizers(metadata, sqlParser, featuresConfig, true); - return createPlan(session, sql, planOptimizers.get(), stage); + return createPlan(session, sql, featuresConfig, planOptimizers.get(), stage); } - public Plan createPlan(Session session, @Language("SQL") String sql, List optimizers) + public Plan createPlan(Session session, @Language("SQL") String sql, FeaturesConfig featuresConfig, List optimizers) { - return createPlan(session, sql, optimizers, LogicalPlanner.Stage.OPTIMIZED_AND_VALIDATED); + return createPlan(session, sql, featuresConfig, optimizers, LogicalPlanner.Stage.OPTIMIZED_AND_VALIDATED); } - public Plan createPlan(Session session, @Language("SQL") String sql, List optimizers, LogicalPlanner.Stage stage) + public Plan createPlan(Session session, @Language("SQL") String sql, FeaturesConfig featuresConfig, List optimizers, LogicalPlanner.Stage stage) { Statement wrapped = sqlParser.createStatement(sql); Statement statement = unwrapExecuteStatement(wrapped, sqlParser, session); @@ -736,7 +738,7 @@ public OperatorFactory duplicate() }; } - public static OperatorFactory createHashProjectOperator(int operatorId, PlanNodeId planNodeId, List columnTypes) + public OperatorFactory createHashProjectOperator(int operatorId, PlanNodeId planNodeId, List columnTypes) { ImmutableList.Builder projectionFunctions = ImmutableList.builder(); for (int i = 0; i < columnTypes.size(); i++) { diff --git a/presto-main/src/main/java/com/facebook/presto/testing/TestingEventListenerManager.java b/presto-main/src/main/java/com/facebook/presto/testing/TestingEventListenerManager.java index 687e84caef97..ce820a5f4ffd 100644 --- a/presto-main/src/main/java/com/facebook/presto/testing/TestingEventListenerManager.java +++ b/presto-main/src/main/java/com/facebook/presto/testing/TestingEventListenerManager.java @@ -36,7 +36,6 @@ public void addEventListenerFactory(EventListenerFactory eventListenerFactory) configuredEventListener.set(Optional.of(eventListenerFactory.create(ImmutableMap.of()))); } - @Override public void queryCompleted(QueryCompletedEvent queryCompletedEvent) { if (configuredEventListener.get().isPresent()) { @@ -44,7 +43,6 @@ public void queryCompleted(QueryCompletedEvent queryCompletedEvent) } } - @Override public void queryCreated(QueryCreatedEvent queryCreatedEvent) { if (configuredEventListener.get().isPresent()) { @@ -52,7 +50,6 @@ public void queryCreated(QueryCreatedEvent queryCreatedEvent) } } - @Override public void splitCompleted(SplitCompletedEvent splitCompletedEvent) { if (configuredEventListener.get().isPresent()) { diff --git a/presto-main/src/main/java/com/facebook/presto/testing/TestingSplit.java b/presto-main/src/main/java/com/facebook/presto/testing/TestingSplit.java index 9ec63a461fe3..537f0227e7dd 100644 --- a/presto-main/src/main/java/com/facebook/presto/testing/TestingSplit.java +++ b/presto-main/src/main/java/com/facebook/presto/testing/TestingSplit.java @@ -36,12 +36,12 @@ public static TestingSplit createLocalSplit() public static TestingSplit createEmptySplit() { - return new TestingSplit(false, ImmutableList.of()); + return new TestingSplit(false, ImmutableList.of()); } public static TestingSplit createRemoteSplit() { - return new TestingSplit(true, ImmutableList.of()); + return new TestingSplit(true, ImmutableList.of()); } @JsonCreator diff --git a/presto-main/src/main/java/com/facebook/presto/transaction/InternalConnector.java b/presto-main/src/main/java/com/facebook/presto/transaction/InternalConnector.java index e9037fc5aef1..d3ce5edc7a45 100644 --- a/presto-main/src/main/java/com/facebook/presto/transaction/InternalConnector.java +++ b/presto-main/src/main/java/com/facebook/presto/transaction/InternalConnector.java @@ -20,7 +20,6 @@ public interface InternalConnector extends Connector { - @Override default ConnectorTransactionHandle beginTransaction(IsolationLevel isolationLevel, boolean readOnly) { throw new UnsupportedOperationException(); diff --git a/presto-main/src/main/java/com/facebook/presto/type/BigintOperators.java b/presto-main/src/main/java/com/facebook/presto/type/BigintOperators.java index 4bf243712e65..208c823703a6 100644 --- a/presto-main/src/main/java/com/facebook/presto/type/BigintOperators.java +++ b/presto-main/src/main/java/com/facebook/presto/type/BigintOperators.java @@ -205,20 +205,6 @@ public static long saturatedFloorCastToInteger(@SqlType(StandardTypes.BIGINT) lo return Ints.saturatedCast(value); } - @ScalarOperator(SATURATED_FLOOR_CAST) - @SqlType(StandardTypes.SMALLINT) - public static long saturatedFloorCastToSmallint(@SqlType(StandardTypes.BIGINT) long value) - { - return Shorts.saturatedCast(value); - } - - @ScalarOperator(SATURATED_FLOOR_CAST) - @SqlType(StandardTypes.TINYINT) - public static long saturatedFloorCastToTinyint(@SqlType(StandardTypes.BIGINT) long value) - { - return SignedBytes.saturatedCast(value); - } - @ScalarOperator(CAST) @SqlType(StandardTypes.SMALLINT) public static long castToSmallint(@SqlType(StandardTypes.BIGINT) long value) diff --git a/presto-main/src/main/java/com/facebook/presto/type/CharOperators.java b/presto-main/src/main/java/com/facebook/presto/type/CharOperators.java index fc50589aa6b7..bf61bf973223 100644 --- a/presto-main/src/main/java/com/facebook/presto/type/CharOperators.java +++ b/presto-main/src/main/java/com/facebook/presto/type/CharOperators.java @@ -30,7 +30,6 @@ import static com.facebook.presto.spi.function.OperatorType.LESS_THAN; import static com.facebook.presto.spi.function.OperatorType.LESS_THAN_OR_EQUAL; import static com.facebook.presto.spi.function.OperatorType.NOT_EQUAL; -import static com.facebook.presto.spi.type.Chars.compareChars; public final class CharOperators { @@ -57,7 +56,7 @@ public static boolean notEqual(@SqlType("char(x)") Slice left, @SqlType("char(x) @SqlType(StandardTypes.BOOLEAN) public static boolean lessThan(@SqlType("char(x)") Slice left, @SqlType("char(x)") Slice right) { - return compareChars(left, right) < 0; + return left.compareTo(right) < 0; } @LiteralParameters({"x"}) @@ -65,7 +64,7 @@ public static boolean lessThan(@SqlType("char(x)") Slice left, @SqlType("char(x) @SqlType(StandardTypes.BOOLEAN) public static boolean lessThanOrEqual(@SqlType("char(x)") Slice left, @SqlType("char(x)") Slice right) { - return compareChars(left, right) <= 0; + return left.compareTo(right) <= 0; } @LiteralParameters({"x"}) @@ -73,7 +72,7 @@ public static boolean lessThanOrEqual(@SqlType("char(x)") Slice left, @SqlType(" @SqlType(StandardTypes.BOOLEAN) public static boolean greaterThan(@SqlType("char(x)") Slice left, @SqlType("char(x)") Slice right) { - return compareChars(left, right) > 0; + return left.compareTo(right) > 0; } @LiteralParameters({"x"}) @@ -81,7 +80,7 @@ public static boolean greaterThan(@SqlType("char(x)") Slice left, @SqlType("char @SqlType(StandardTypes.BOOLEAN) public static boolean greaterThanOrEqual(@SqlType("char(x)") Slice left, @SqlType("char(x)") Slice right) { - return compareChars(left, right) >= 0; + return left.compareTo(right) >= 0; } @LiteralParameters({"x"}) @@ -89,7 +88,7 @@ public static boolean greaterThanOrEqual(@SqlType("char(x)") Slice left, @SqlTyp @SqlType(StandardTypes.BOOLEAN) public static boolean between(@SqlType("char(x)") Slice value, @SqlType("char(x)") Slice min, @SqlType("char(x)") Slice max) { - return compareChars(min, value) <= 0 && compareChars(value, max) <= 0; + return min.compareTo(value) <= 0 && value.compareTo(max) <= 0; } @LiteralParameters("x") diff --git a/presto-main/src/main/java/com/facebook/presto/type/DecimalCasts.java b/presto-main/src/main/java/com/facebook/presto/type/DecimalCasts.java index d1f059ad9dc2..412ce8c594fa 100644 --- a/presto-main/src/main/java/com/facebook/presto/type/DecimalCasts.java +++ b/presto-main/src/main/java/com/facebook/presto/type/DecimalCasts.java @@ -26,6 +26,7 @@ import com.fasterxml.jackson.core.JsonParser; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import com.google.common.primitives.Ints; import com.google.common.primitives.Shorts; import com.google.common.primitives.SignedBytes; import io.airlift.slice.DynamicSliceOutput; @@ -38,6 +39,7 @@ import java.io.IOException; import java.math.BigDecimal; import java.math.BigInteger; +import java.math.RoundingMode; import static com.facebook.presto.metadata.FunctionKind.SCALAR; import static com.facebook.presto.operator.scalar.JsonOperators.JSON_FACTORY; @@ -62,13 +64,12 @@ import static com.facebook.presto.util.JsonUtil.createJsonGenerator; import static com.facebook.presto.util.JsonUtil.createJsonParser; import static com.facebook.presto.util.Types.checkType; -import static java.lang.Double.isFinite; import static java.lang.Float.floatToRawIntBits; import static java.lang.Float.intBitsToFloat; import static java.lang.Math.multiplyExact; import static java.lang.String.format; +import static java.math.BigDecimal.ROUND_HALF_UP; import static java.math.BigInteger.ZERO; -import static java.math.RoundingMode.HALF_UP; import static java.nio.charset.StandardCharsets.UTF_8; public final class DecimalCasts @@ -109,10 +110,10 @@ private static SqlScalarFunction castFunctionFromDecimalTo(TypeSignature to, Str long scale = context.getLiteral("scale"); Number tenToScale; if (isShortDecimal(context.getParameterTypes().get(0))) { - tenToScale = longTenToNth(intScale(scale)); + tenToScale = longTenToNth((int) scale); } else { - tenToScale = bigIntegerTenToNth(intScale(scale)); + tenToScale = bigIntegerTenToNth((int) scale); } return ImmutableList.of(precision, scale, tenToScale); }) @@ -184,14 +185,16 @@ public static long shortDecimalToBigint(long decimal, long precision, long scale if (decimal >= 0) { return (decimal + tenToScale / 2) / tenToScale; } - return -((-decimal + tenToScale / 2) / tenToScale); + else { + return -((-decimal + tenToScale / 2) / tenToScale); + } } @UsedByGeneratedCode public static long longDecimalToBigint(Slice decimal, long precision, long scale, BigInteger tenToScale) { - BigDecimal bigDecimal = new BigDecimal(decodeUnscaledValue(decimal), intScale(scale)); - bigDecimal = bigDecimal.setScale(0, HALF_UP); + BigDecimal bigDecimal = new BigDecimal(decodeUnscaledValue(decimal), (int) scale); + bigDecimal = bigDecimal.setScale(0, RoundingMode.HALF_UP); try { return bigDecimal.longValueExact(); } @@ -205,7 +208,7 @@ public static long bigintToShortDecimal(long value, long precision, long scale, { try { long decimal = multiplyExact(value, tenToScale); - if (overflows(decimal, intScale(precision))) { + if (overflows(decimal, (int) precision)) { throw new PrestoException(INVALID_CAST_ARGUMENT, format("Cannot cast BIGINT '%s' to DECIMAL(%s, %s)", value, precision, scale)); } return decimal; @@ -219,7 +222,7 @@ public static long bigintToShortDecimal(long value, long precision, long scale, public static Slice bigintToLongDecimal(long value, long precision, long scale, BigInteger tenToScale) { BigInteger decimalBigInteger = BigInteger.valueOf(value).multiply(tenToScale); - if (overflows(decimalBigInteger, intScale(precision))) { + if (overflows(decimalBigInteger, (int) precision)) { throw new PrestoException(INVALID_CAST_ARGUMENT, format("Cannot cast BIGINT '%s' to DECIMAL(%s, %s)", value, precision, scale)); } return encodeUnscaledValue(decimalBigInteger); @@ -245,8 +248,8 @@ public static long shortDecimalToInteger(long decimal, long precision, long scal @UsedByGeneratedCode public static long longDecimalToInteger(Slice decimal, long precision, long scale, BigInteger tenToScale) { - BigDecimal bigDecimal = new BigDecimal(decodeUnscaledValue(decimal), intScale(scale)); - bigDecimal = bigDecimal.setScale(0, HALF_UP); + BigDecimal bigDecimal = new BigDecimal(decodeUnscaledValue(decimal), (int) scale); + bigDecimal = bigDecimal.setScale(0, RoundingMode.HALF_UP); try { return bigDecimal.intValueExact(); } @@ -260,7 +263,7 @@ public static long integerToShortDecimal(long value, long precision, long scale, { try { long decimal = multiplyExact(value, tenToScale); - if (overflows(decimal, intScale(precision))) { + if (overflows(decimal, (int) precision)) { throw new PrestoException(INVALID_CAST_ARGUMENT, format("Cannot cast INTEGER '%s' to DECIMAL(%s, %s)", value, precision, scale)); } return decimal; @@ -274,7 +277,7 @@ public static long integerToShortDecimal(long value, long precision, long scale, public static Slice integerToLongDecimal(long value, long precision, long scale, BigInteger tenToScale) { BigInteger decimalBigInteger = BigInteger.valueOf(value).multiply(tenToScale); - if (overflows(decimalBigInteger, intScale(precision))) { + if (overflows(decimalBigInteger, (int) precision)) { throw new PrestoException(INVALID_CAST_ARGUMENT, format("Cannot cast INTEGER '%s' to DECIMAL(%s, %s)", value, precision, scale)); } return encodeUnscaledValue(decimalBigInteger); @@ -300,8 +303,8 @@ public static long shortDecimalToSmallint(long decimal, long precision, long sca @UsedByGeneratedCode public static long longDecimalToSmallint(Slice decimal, long precision, long scale, BigInteger tenToScale) { - BigDecimal bigDecimal = new BigDecimal(decodeUnscaledValue(decimal), intScale(scale)); - bigDecimal = bigDecimal.setScale(0, HALF_UP); + BigDecimal bigDecimal = new BigDecimal(decodeUnscaledValue(decimal), (int) scale); + bigDecimal = bigDecimal.setScale(0, RoundingMode.HALF_UP); try { return bigDecimal.shortValueExact(); } @@ -315,7 +318,7 @@ public static long smallintToShortDecimal(long value, long precision, long scale { try { long decimal = multiplyExact(value, tenToScale); - if (overflows(decimal, intScale(precision))) { + if (overflows(decimal, (int) precision)) { throw new PrestoException(INVALID_CAST_ARGUMENT, format("Cannot cast SMALLINT '%s' to DECIMAL(%s, %s)", value, precision, scale)); } return decimal; @@ -329,7 +332,7 @@ public static long smallintToShortDecimal(long value, long precision, long scale public static Slice smallintToLongDecimal(long value, long precision, long scale, BigInteger tenToScale) { BigInteger decimalBigInteger = BigInteger.valueOf(value).multiply(tenToScale); - if (overflows(decimalBigInteger, intScale(precision))) { + if (overflows(decimalBigInteger, (int) precision)) { throw new PrestoException(INVALID_CAST_ARGUMENT, format("Cannot cast SMALLINT '%s' to DECIMAL(%s, %s)", value, precision, scale)); } return encodeUnscaledValue(decimalBigInteger); @@ -355,8 +358,8 @@ public static long shortDecimalToTinyint(long decimal, long precision, long scal @UsedByGeneratedCode public static long longDecimalToTinyint(Slice decimal, long precision, long scale, BigInteger tenToScale) { - BigDecimal bigDecimal = new BigDecimal(decodeUnscaledValue(decimal), intScale(scale)); - bigDecimal = bigDecimal.setScale(0, HALF_UP); + BigDecimal bigDecimal = new BigDecimal(decodeUnscaledValue(decimal), (int) scale); + bigDecimal = bigDecimal.setScale(0, RoundingMode.HALF_UP); try { return bigDecimal.byteValueExact(); } @@ -370,7 +373,7 @@ public static long tinyintToShortDecimal(long value, long precision, long scale, { try { long decimal = multiplyExact(value, tenToScale); - if (overflows(decimal, intScale(precision))) { + if (overflows(decimal, (int) precision)) { throw new PrestoException(INVALID_CAST_ARGUMENT, format("Cannot cast TINYINT '%s' to DECIMAL(%s, %s)", value, precision, scale)); } return decimal; @@ -384,7 +387,7 @@ public static long tinyintToShortDecimal(long value, long precision, long scale, public static Slice tinyintToLongDecimal(long value, long precision, long scale, BigInteger tenToScale) { BigInteger decimalBigInteger = BigInteger.valueOf(value).multiply(tenToScale); - if (overflows(decimalBigInteger, intScale(precision))) { + if (overflows(decimalBigInteger, (int) precision)) { throw new PrestoException(INVALID_CAST_ARGUMENT, format("Cannot cast TINYINT '%s' to DECIMAL(%s, %s)", value, precision, scale)); } return encodeUnscaledValue(decimalBigInteger); @@ -400,7 +403,7 @@ public static double shortDecimalToDouble(long decimal, long precision, long sca public static double longDecimalToDouble(Slice decimal, long precision, long scale, BigInteger tenToScale) { BigInteger decimalBigInteger = decodeUnscaledValue(decimal); - BigDecimal bigDecimal = new BigDecimal(decimalBigInteger, intScale(scale)); + BigDecimal bigDecimal = new BigDecimal(decimalBigInteger, (int) scale); return bigDecimal.doubleValue(); } @@ -414,15 +417,15 @@ public static long shortDecimalToReal(long decimal, long precision, long scale, public static long longDecimalToReal(Slice decimal, long precision, long scale, BigInteger tenToScale) { BigInteger decimalBigInteger = decodeUnscaledValue(decimal); - BigDecimal bigDecimal = new BigDecimal(decimalBigInteger, intScale(scale)); + BigDecimal bigDecimal = new BigDecimal(decimalBigInteger, (int) scale); return floatToRawIntBits(bigDecimal.floatValue()); } @UsedByGeneratedCode public static long doubleToShortDecimal(double value, long precision, long scale, long tenToScale) { - BigDecimal decimal = doubleToBigDecimal(value, precision, scale); - decimal = decimal.setScale(intScale(scale), HALF_UP); + BigDecimal decimal = new BigDecimal(value); + decimal = decimal.setScale((int) scale, ROUND_HALF_UP); if (overflows(decimal, precision)) { throw new PrestoException(INVALID_CAST_ARGUMENT, format("Cannot cast DOUBLE '%s' to DECIMAL(%s, %s)", value, precision, scale)); } @@ -432,8 +435,8 @@ public static long doubleToShortDecimal(double value, long precision, long scale @UsedByGeneratedCode public static Slice doubleToLongDecimal(double value, long precision, long scale, BigInteger tenToScale) { - BigDecimal decimal = doubleToBigDecimal(value, precision, scale); - decimal = decimal.setScale(intScale(scale), HALF_UP); + BigDecimal decimal = new BigDecimal(value); + decimal = decimal.setScale((int) scale, ROUND_HALF_UP); if (overflows(decimal, precision)) { throw new PrestoException(INVALID_CAST_ARGUMENT, format("Cannot cast DOUBLE '%s' to DECIMAL(%s, %s)", value, precision, scale)); } @@ -444,10 +447,10 @@ public static Slice doubleToLongDecimal(double value, long precision, long scale @UsedByGeneratedCode public static long realToShortDecimal(long value, long precision, long scale, long tenToScale) { - BigDecimal decimal = doubleToBigDecimal(intBitsToFloat(intScale(value)), precision, scale); - decimal = decimal.setScale(intScale(scale), HALF_UP); + BigDecimal decimal = new BigDecimal(intBitsToFloat((int) value)); + decimal = decimal.setScale((int) scale, ROUND_HALF_UP); if (overflows(decimal, precision)) { - throw new PrestoException(INVALID_CAST_ARGUMENT, format("Cannot cast REAL '%s' to DECIMAL(%s, %s)", intBitsToFloat(intScale(value)), precision, scale)); + throw new PrestoException(INVALID_CAST_ARGUMENT, format("Cannot cast REAL '%s' to DECIMAL(%s, %s)", intBitsToFloat((int) value), precision, scale)); } return decimal.unscaledValue().longValue(); } @@ -455,33 +458,25 @@ public static long realToShortDecimal(long value, long precision, long scale, lo @UsedByGeneratedCode public static Slice realToLongDecimal(long value, long precision, long scale, BigInteger tenToScale) { - BigDecimal decimal = doubleToBigDecimal(intBitsToFloat(intScale(value)), precision, scale); - decimal = decimal.setScale(intScale(scale), HALF_UP); + BigDecimal decimal = new BigDecimal(intBitsToFloat((int) value)); + decimal = decimal.setScale((int) scale, ROUND_HALF_UP); if (overflows(decimal, precision)) { - throw new PrestoException(INVALID_CAST_ARGUMENT, format("Cannot cast REAL '%s' to DECIMAL(%s, %s)", intBitsToFloat(intScale(value)), precision, scale)); + throw new PrestoException(INVALID_CAST_ARGUMENT, format("Cannot cast REAL '%s' to DECIMAL(%s, %s)", intBitsToFloat((int) value), precision, scale)); } BigInteger decimalBigInteger = decimal.unscaledValue(); return encodeUnscaledValue(decimalBigInteger); } - private static BigDecimal doubleToBigDecimal(double value, long precision, long scale) - { - if (!isFinite(value)) { - throw new PrestoException(INVALID_CAST_ARGUMENT, format("Cannot cast floating point %s to DECIMAL(%s, %s)", value, precision, scale)); - } - return new BigDecimal(value); - } - @UsedByGeneratedCode public static Slice shortDecimalToVarchar(long decimal, long precision, long scale, long tenToScale) { - return Slices.copiedBuffer(Decimals.toString(decimal, intScale(scale)), UTF_8); + return Slices.copiedBuffer(Decimals.toString(decimal, (int) scale), UTF_8); } @UsedByGeneratedCode public static Slice longDecimalToVarchar(Slice decimal, long precision, long scale, BigInteger tenToScale) { - return Slices.copiedBuffer(Decimals.toString(decimal, intScale(scale)), UTF_8); + return Slices.copiedBuffer(Decimals.toString(decimal, (int) scale), UTF_8); } @UsedByGeneratedCode @@ -489,7 +484,7 @@ public static long varcharToShortDecimal(Slice value, long precision, long scale { try { String stringValue = value.toString(UTF_8); - BigDecimal decimal = new BigDecimal(stringValue).setScale(intScale(scale), HALF_UP); + BigDecimal decimal = new BigDecimal(stringValue).setScale((int) scale, ROUND_HALF_UP); if (overflows(decimal, precision)) { throw new PrestoException(INVALID_CAST_ARGUMENT, format("Cannot cast VARCHAR '%s' to DECIMAL(%s, %s)", stringValue, precision, scale)); } @@ -504,7 +499,7 @@ public static long varcharToShortDecimal(Slice value, long precision, long scale public static Slice varcharToLongDecimal(Slice value, long precision, long scale, BigInteger tenToScale) { String stringValue = value.toString(UTF_8); - BigDecimal decimal = new BigDecimal(stringValue).setScale(intScale(scale), HALF_UP); + BigDecimal decimal = new BigDecimal(stringValue).setScale((int) scale, ROUND_HALF_UP); if (overflows(decimal, precision)) { throw new PrestoException(INVALID_CAST_ARGUMENT, format("Cannot cast VARCHAR '%s' to DECIMAL(%s, %s)", stringValue, precision, scale)); } @@ -515,14 +510,14 @@ public static Slice varcharToLongDecimal(Slice value, long precision, long scale public static Slice shortDecimalToJson(long decimal, long precision, long scale, long tenToScale) throws IOException { - return decimalToJson(BigDecimal.valueOf(decimal, intScale(scale))); + return decimalToJson(BigDecimal.valueOf(decimal, (int) scale)); } @UsedByGeneratedCode public static Slice longDecimalToJson(Slice decimal, long precision, long scale, BigInteger tenToScale) throws IOException { - return decimalToJson(new BigDecimal(decodeUnscaledValue(decimal), intScale(scale))); + return decimalToJson(new BigDecimal(Decimals.decodeUnscaledValue(decimal), Ints.checkedCast(scale))); } private static Slice decimalToJson(BigDecimal bigDecimal) @@ -547,7 +542,7 @@ public static Slice jsonToLongDecimal(Slice json, long precision, long scale, Bi if (bigDecimal == null) { return null; } - return encodeUnscaledValue(bigDecimal.unscaledValue()); + return Decimals.encodeUnscaledValue(bigDecimal.unscaledValue()); } @UsedByGeneratedCode @@ -570,18 +565,18 @@ private static BigDecimal jsonToDecimal(Slice json, long precision, long scale) break; case VALUE_STRING: result = new BigDecimal(parser.getText()); - result = result.setScale(intScale(scale), HALF_UP); + result = result.setScale((int) scale, ROUND_HALF_UP); break; case VALUE_NUMBER_FLOAT: case VALUE_NUMBER_INT: result = parser.getDecimalValue(); - result = result.setScale(intScale(scale), HALF_UP); + result = result.setScale((int) scale, ROUND_HALF_UP); break; case VALUE_TRUE: - result = BigDecimal.ONE.setScale(intScale(scale), HALF_UP); + result = BigDecimal.ONE.setScale((int) scale, ROUND_HALF_UP); break; case VALUE_FALSE: - result = BigDecimal.ZERO.setScale(intScale(scale), HALF_UP); + result = BigDecimal.ZERO.setScale((int) scale, ROUND_HALF_UP); break; default: throw new PrestoException(INVALID_CAST_ARGUMENT, format("Cannot cast '%s' to DECIMAL(%s,%s)", json.toStringUtf8(), precision, scale)); @@ -597,10 +592,4 @@ private static BigDecimal jsonToDecimal(Slice json, long precision, long scale) throw new PrestoException(INVALID_CAST_ARGUMENT, format("Cannot cast '%s' to DECIMAL(%s,%s)", json.toStringUtf8(), precision, scale)); } } - - @SuppressWarnings("NumericCastThatLosesPrecision") - private static int intScale(long scale) - { - return (int) scale; - } } diff --git a/presto-main/src/main/java/com/facebook/presto/type/DecimalSaturatedFloorCasts.java b/presto-main/src/main/java/com/facebook/presto/type/DecimalSaturatedFloorCasts.java index 2ad47acc9f23..042c174b07ba 100644 --- a/presto-main/src/main/java/com/facebook/presto/type/DecimalSaturatedFloorCasts.java +++ b/presto-main/src/main/java/com/facebook/presto/type/DecimalSaturatedFloorCasts.java @@ -16,7 +16,6 @@ import com.facebook.presto.annotation.UsedByGeneratedCode; import com.facebook.presto.metadata.Signature; import com.facebook.presto.metadata.SqlScalarFunction; -import com.facebook.presto.spi.type.Type; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.primitives.Ints; @@ -27,17 +26,10 @@ import static com.facebook.presto.metadata.FunctionKind.SCALAR; import static com.facebook.presto.spi.function.OperatorType.SATURATED_FLOOR_CAST; -import static com.facebook.presto.spi.type.BigintType.BIGINT; import static com.facebook.presto.spi.type.Decimals.bigIntegerTenToNth; import static com.facebook.presto.spi.type.Decimals.decodeUnscaledValue; import static com.facebook.presto.spi.type.Decimals.encodeUnscaledValue; -import static com.facebook.presto.spi.type.DoubleType.DOUBLE; -import static com.facebook.presto.spi.type.IntegerType.INTEGER; -import static com.facebook.presto.spi.type.RealType.REAL; -import static com.facebook.presto.spi.type.SmallintType.SMALLINT; -import static com.facebook.presto.spi.type.TinyintType.TINYINT; import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature; -import static java.lang.Float.intBitsToFloat; import static java.math.BigInteger.ONE; import static java.math.RoundingMode.FLOOR; @@ -90,13 +82,9 @@ public static Slice longDecimalToLongDecimal(Slice value, int sourcePrecision, i private static BigInteger bigintToBigintFloorSaturatedCast(BigInteger value, int sourceScale, int resultPrecision, int resultScale) { - return bigDecimalToBigintFloorSaturatedCast(new BigDecimal(value, sourceScale), resultPrecision, resultScale); - } - - private static BigInteger bigDecimalToBigintFloorSaturatedCast(BigDecimal bigDecimal, int resultPrecision, int resultScale) - { - BigDecimal rescaledValue = bigDecimal.setScale(resultScale, FLOOR); - BigInteger unscaledValue = rescaledValue.unscaledValue(); + BigDecimal bigDecimal = new BigDecimal(value, sourceScale); + bigDecimal = bigDecimal.setScale(resultScale, FLOOR); + BigInteger unscaledValue = bigDecimal.unscaledValue(); BigInteger maxUnscaledValue = bigIntegerTenToNth(resultPrecision).subtract(ONE); if (unscaledValue.compareTo(maxUnscaledValue) > 0) { return maxUnscaledValue; @@ -107,148 +95,4 @@ private static BigInteger bigDecimalToBigintFloorSaturatedCast(BigDecimal bigDec } return unscaledValue; } - - public static final SqlScalarFunction DOUBLE_TO_DECIMAL_SATURATED_FLOOR_CAST = SqlScalarFunction.builder(DecimalSaturatedFloorCasts.class) - .signature(Signature.builder() - .kind(SCALAR) - .operatorType(SATURATED_FLOOR_CAST) - .argumentTypes(DOUBLE.getTypeSignature()) - .returnType(parseTypeSignature("decimal(result_precision,result_scale)", ImmutableSet.of("result_precision", "result_scale"))) - .build() - ) - .implementation(b -> b - .methods("doubleToShortDecimal", "doubleToLongDecimal") - .withExtraParameters((context) -> { - int resultPrecision = Ints.checkedCast(context.getLiteral("result_precision")); - int resultScale = Ints.checkedCast(context.getLiteral("result_scale")); - return ImmutableList.of(resultPrecision, resultScale); - }) - ).build(); - - @UsedByGeneratedCode - public static long doubleToShortDecimal(double value, int resultPrecision, int resultScale) - { - return bigDecimalToBigintFloorSaturatedCast(new BigDecimal(value), resultPrecision, resultScale).longValueExact(); - } - - @UsedByGeneratedCode - public static Slice doubleToLongDecimal(double value, int resultPrecision, int resultScale) - { - return encodeUnscaledValue(bigDecimalToBigintFloorSaturatedCast(new BigDecimal(value), resultPrecision, resultScale)); - } - - public static final SqlScalarFunction REAL_TO_DECIMAL_SATURATED_FLOOR_CAST = SqlScalarFunction.builder(DecimalSaturatedFloorCasts.class) - .signature(Signature.builder() - .kind(SCALAR) - .operatorType(SATURATED_FLOOR_CAST) - .argumentTypes(REAL.getTypeSignature()) - .returnType(parseTypeSignature("decimal(result_precision,result_scale)", ImmutableSet.of("result_precision", "result_scale"))) - .build() - ) - .implementation(b -> b - .methods("realToShortDecimal", "realToLongDecimal") - .withExtraParameters((context) -> { - int resultPrecision = Ints.checkedCast(context.getLiteral("result_precision")); - int resultScale = Ints.checkedCast(context.getLiteral("result_scale")); - return ImmutableList.of(resultPrecision, resultScale); - }) - ).build(); - - @UsedByGeneratedCode - public static long realToShortDecimal(long value, int resultPrecision, int resultScale) - { - return bigDecimalToBigintFloorSaturatedCast(new BigDecimal(intBitsToFloat((int) value)), resultPrecision, resultScale).longValueExact(); - } - - @UsedByGeneratedCode - public static Slice realToLongDecimal(long value, int resultPrecision, int resultScale) - { - return encodeUnscaledValue(bigDecimalToBigintFloorSaturatedCast(new BigDecimal(intBitsToFloat((int) value)), resultPrecision, resultScale)); - } - - public static final SqlScalarFunction DECIMAL_TO_BIGINT_SATURATED_FLOOR_CAST = decimalToGenericIntegerTypeSaturatedFloorCast(BIGINT, Long.MIN_VALUE, Long.MAX_VALUE); - public static final SqlScalarFunction DECIMAL_TO_INTEGER_SATURATED_FLOOR_CAST = decimalToGenericIntegerTypeSaturatedFloorCast(INTEGER, Integer.MIN_VALUE, Integer.MAX_VALUE); - public static final SqlScalarFunction DECIMAL_TO_SMALLINT_SATURATED_FLOOR_CAST = decimalToGenericIntegerTypeSaturatedFloorCast(SMALLINT, Short.MIN_VALUE, Short.MAX_VALUE); - public static final SqlScalarFunction DECIMAL_TO_TINYINT_SATURATED_FLOOR_CAST = decimalToGenericIntegerTypeSaturatedFloorCast(TINYINT, Byte.MIN_VALUE, Byte.MAX_VALUE); - - private static SqlScalarFunction decimalToGenericIntegerTypeSaturatedFloorCast(Type type, long minValue, long maxValue) - { - return SqlScalarFunction.builder(DecimalSaturatedFloorCasts.class) - .signature(Signature.builder() - .kind(SCALAR) - .operatorType(SATURATED_FLOOR_CAST) - .argumentTypes(parseTypeSignature("decimal(source_precision,source_scale)", ImmutableSet.of("source_precision", "source_scale"))) - .returnType(type.getTypeSignature()) - .build() - ) - .implementation(b -> b - .methods("shortDecimalToGenericIntegerType", "longDecimalToGenericIntegerType") - .withExtraParameters((context) -> { - int sourceScale = Ints.checkedCast(context.getLiteral("source_scale")); - return ImmutableList.of(sourceScale, minValue, maxValue); - }) - ).build(); - } - - @UsedByGeneratedCode - public static long shortDecimalToGenericIntegerType(long value, int sourceScale, long minValue, long maxValue) - { - return bigIntegerDecimalToGenericIntegerType(BigInteger.valueOf(value), sourceScale, minValue, maxValue); - } - - @UsedByGeneratedCode - public static long longDecimalToGenericIntegerType(Slice value, int sourceScale, long minValue, long maxValue) - { - return bigIntegerDecimalToGenericIntegerType(decodeUnscaledValue(value), sourceScale, minValue, maxValue); - } - - private static long bigIntegerDecimalToGenericIntegerType(BigInteger bigInteger, int sourceScale, long minValue, long maxValue) - { - BigDecimal bigDecimal = new BigDecimal(bigInteger, sourceScale); - BigInteger unscaledValue = bigDecimal.setScale(0, FLOOR).unscaledValue(); - if (unscaledValue.compareTo(BigInteger.valueOf(maxValue)) > 0) { - return maxValue; - } - if (unscaledValue.compareTo(BigInteger.valueOf(minValue)) < 0) { - return minValue; - } - return unscaledValue.longValueExact(); - } - - public static final SqlScalarFunction BIGINT_TO_DECIMAL_SATURATED_FLOOR_CAST = genericIntegerTypeToDecimalSaturatedFloorCast(BIGINT); - public static final SqlScalarFunction INTEGER_TO_DECIMAL_SATURATED_FLOOR_CAST = genericIntegerTypeToDecimalSaturatedFloorCast(INTEGER); - public static final SqlScalarFunction SMALLINT_TO_DECIMAL_SATURATED_FLOOR_CAST = genericIntegerTypeToDecimalSaturatedFloorCast(SMALLINT); - public static final SqlScalarFunction TINYINT_TO_DECIMAL_SATURATED_FLOOR_CAST = genericIntegerTypeToDecimalSaturatedFloorCast(TINYINT); - - private static SqlScalarFunction genericIntegerTypeToDecimalSaturatedFloorCast(Type integerType) - { - return SqlScalarFunction.builder(DecimalSaturatedFloorCasts.class) - .signature(Signature.builder() - .kind(SCALAR) - .operatorType(SATURATED_FLOOR_CAST) - .argumentTypes(integerType.getTypeSignature()) - .returnType(parseTypeSignature("decimal(result_precision,result_scale)", ImmutableSet.of("result_precision", "result_scale"))) - .build() - ) - .implementation(b -> b - .methods("genericIntegerTypeToShortDecimal", "genericIntegerTypeToLongDecimal") - .withExtraParameters((context) -> { - int resultPrecision = Ints.checkedCast(context.getLiteral("result_precision")); - int resultScale = Ints.checkedCast(context.getLiteral("result_scale")); - return ImmutableList.of(resultPrecision, resultScale); - }) - ).build(); - } - - @UsedByGeneratedCode - public static long genericIntegerTypeToShortDecimal(long value, int resultPrecision, int resultScale) - { - return bigDecimalToBigintFloorSaturatedCast(BigDecimal.valueOf(value), resultPrecision, resultScale).longValueExact(); - } - - @UsedByGeneratedCode - public static Slice genericIntegerTypeToLongDecimal(long value, int resultPrecision, int resultScale) - { - return encodeUnscaledValue(bigDecimalToBigintFloorSaturatedCast(BigDecimal.valueOf(value), resultPrecision, resultScale)); - } } diff --git a/presto-main/src/main/java/com/facebook/presto/type/DoubleOperators.java b/presto-main/src/main/java/com/facebook/presto/type/DoubleOperators.java index 0345e2ac2cdf..3a35149ab29a 100644 --- a/presto-main/src/main/java/com/facebook/presto/type/DoubleOperators.java +++ b/presto-main/src/main/java/com/facebook/presto/type/DoubleOperators.java @@ -46,7 +46,6 @@ import static com.facebook.presto.spi.function.OperatorType.NOT_EQUAL; import static com.facebook.presto.spi.function.OperatorType.SATURATED_FLOOR_CAST; import static com.facebook.presto.spi.function.OperatorType.SUBTRACT; -import static com.google.common.base.Preconditions.checkState; import static io.airlift.slice.Slices.utf8Slice; import static java.lang.Double.doubleToLongBits; import static java.lang.Float.floatToRawIntBits; @@ -56,13 +55,9 @@ public final class DoubleOperators { private static final double MIN_LONG_AS_DOUBLE = -0x1p63; - private static final double MAX_LONG_PLUS_ONE_AS_DOUBLE = 0x1p63; - private static final double MIN_INTEGER_AS_DOUBLE = -0x1p31; - private static final double MAX_INTEGER_PLUS_ONE_AS_DOUBLE = 0x1p31; - private static final double MIN_SHORT_AS_DOUBLE = -0x1p15; - private static final double MAX_SHORT_PLUS_ONE_AS_DOUBLE = 0x1p15; - private static final double MIN_BYTE_AS_DOUBLE = -0x1p7; - private static final double MAX_BYTE_PLUS_ONE_AS_DOUBLE = 0x1p7; + private static final double MAX_LONG_AS_DOUBLE_PLUS_ONE = 0x1p63; + private static final double MIN_INT_AS_DOUBLE = -0x1p31; + private static final double MAX_INT_AS_DOUBLE = 0x1p31 - 1.0; private DoubleOperators() { @@ -243,65 +238,30 @@ public static long hashCode(@SqlType(StandardTypes.DOUBLE) double value) return AbstractLongType.hash(doubleToLongBits(value)); } - @ScalarOperator(SATURATED_FLOOR_CAST) - @SqlType(StandardTypes.REAL) - public static strictfp long saturatedFloorCastToFloat(@SqlType(StandardTypes.DOUBLE) double value) - { - float result; - float minFloat = -1.0f * Float.MAX_VALUE; - if (value <= minFloat) { - result = minFloat; - } - else if (value >= Float.MAX_VALUE) { - result = Float.MAX_VALUE; - } - else { - result = (float) value; - if (result > value) { - result = Math.nextDown(result); - } - checkState(result <= value); - } - return floatToRawIntBits(result); - } - @ScalarOperator(SATURATED_FLOOR_CAST) @SqlType(StandardTypes.BIGINT) public static long saturatedFloorCastToBigint(@SqlType(StandardTypes.DOUBLE) double value) { - return saturatedFloorCastToLong(value, Long.MIN_VALUE, MIN_LONG_AS_DOUBLE, Long.MAX_VALUE, MAX_LONG_PLUS_ONE_AS_DOUBLE); + if (value <= MIN_LONG_AS_DOUBLE) { + return Long.MIN_VALUE; + } + if (MAX_LONG_AS_DOUBLE_PLUS_ONE - value <= 1) { + return Long.MAX_VALUE; + } + return DoubleMath.roundToLong(value, FLOOR); } @ScalarOperator(SATURATED_FLOOR_CAST) @SqlType(StandardTypes.INTEGER) public static long saturatedFloorCastToInteger(@SqlType(StandardTypes.DOUBLE) double value) { - return saturatedFloorCastToLong(value, Integer.MIN_VALUE, MIN_INTEGER_AS_DOUBLE, Integer.MAX_VALUE, MAX_INTEGER_PLUS_ONE_AS_DOUBLE); - } - - @ScalarOperator(SATURATED_FLOOR_CAST) - @SqlType(StandardTypes.SMALLINT) - public static long saturatedFloorCastToSmallint(@SqlType(StandardTypes.DOUBLE) double value) - { - return saturatedFloorCastToLong(value, Short.MIN_VALUE, MIN_SHORT_AS_DOUBLE, Short.MAX_VALUE, MAX_SHORT_PLUS_ONE_AS_DOUBLE); - } - - @ScalarOperator(SATURATED_FLOOR_CAST) - @SqlType(StandardTypes.TINYINT) - public static long saturatedFloorCastToTinyint(@SqlType(StandardTypes.DOUBLE) double value) - { - return saturatedFloorCastToLong(value, Byte.MIN_VALUE, MIN_BYTE_AS_DOUBLE, Byte.MAX_VALUE, MAX_BYTE_PLUS_ONE_AS_DOUBLE); - } - - private static long saturatedFloorCastToLong(double value, long minValue, double minValueAsDouble, long maxValue, double maxValuePlusOneAsDouble) - { - if (value <= minValueAsDouble) { - return minValue; + if (value <= MIN_INT_AS_DOUBLE) { + return Integer.MIN_VALUE; } - if (value + 1 >= maxValuePlusOneAsDouble) { - return maxValue; + if (MAX_INT_AS_DOUBLE - value <= 1) { + return Integer.MAX_VALUE; } - return DoubleMath.roundToLong(value, FLOOR); + return DoubleMath.roundToInt(value, FLOOR); } @ScalarOperator(IS_DISTINCT_FROM) diff --git a/presto-main/src/main/java/com/facebook/presto/type/IntegerOperators.java b/presto-main/src/main/java/com/facebook/presto/type/IntegerOperators.java index b1efa317c331..a7132d2c8d91 100644 --- a/presto-main/src/main/java/com/facebook/presto/type/IntegerOperators.java +++ b/presto-main/src/main/java/com/facebook/presto/type/IntegerOperators.java @@ -41,7 +41,6 @@ import static com.facebook.presto.spi.function.OperatorType.MULTIPLY; import static com.facebook.presto.spi.function.OperatorType.NEGATION; import static com.facebook.presto.spi.function.OperatorType.NOT_EQUAL; -import static com.facebook.presto.spi.function.OperatorType.SATURATED_FLOOR_CAST; import static com.facebook.presto.spi.function.OperatorType.SUBTRACT; import static io.airlift.slice.Slices.utf8Slice; import static java.lang.Float.floatToRawIntBits; @@ -259,18 +258,4 @@ public static boolean isDistinctFrom( } return notEqual(left, right); } - - @ScalarOperator(SATURATED_FLOOR_CAST) - @SqlType(StandardTypes.SMALLINT) - public static long saturatedFloorCastToSmallint(@SqlType(StandardTypes.INTEGER) long value) - { - return Shorts.saturatedCast(value); - } - - @ScalarOperator(SATURATED_FLOOR_CAST) - @SqlType(StandardTypes.TINYINT) - public static long saturatedFloorCastToTinyint(@SqlType(StandardTypes.INTEGER) long value) - { - return SignedBytes.saturatedCast(value); - } } diff --git a/presto-main/src/main/java/com/facebook/presto/type/Re2JRegexp.java b/presto-main/src/main/java/com/facebook/presto/type/Re2JRegexp.java index bbc5e96e95ae..34b7d0dcb08f 100644 --- a/presto-main/src/main/java/com/facebook/presto/type/Re2JRegexp.java +++ b/presto-main/src/main/java/com/facebook/presto/type/Re2JRegexp.java @@ -139,7 +139,7 @@ public Block split(Slice source) return blockBuilder.build(); } - private static void validateGroup(int group, int groupCount) + private void validateGroup(int group, int groupCount) { if (group < 0) { throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "Group cannot be negative"); diff --git a/presto-main/src/main/java/com/facebook/presto/type/RealOperators.java b/presto-main/src/main/java/com/facebook/presto/type/RealOperators.java index 4ab7a4aae9fc..b24ee5e76399 100644 --- a/presto-main/src/main/java/com/facebook/presto/type/RealOperators.java +++ b/presto-main/src/main/java/com/facebook/presto/type/RealOperators.java @@ -21,7 +21,6 @@ import com.facebook.presto.spi.function.SqlType; import com.facebook.presto.spi.type.AbstractIntType; import com.facebook.presto.spi.type.StandardTypes; -import com.google.common.math.DoubleMath; import com.google.common.primitives.Ints; import com.google.common.primitives.Shorts; import com.google.common.primitives.SignedBytes; @@ -43,25 +42,14 @@ import static com.facebook.presto.spi.function.OperatorType.MULTIPLY; import static com.facebook.presto.spi.function.OperatorType.NEGATION; import static com.facebook.presto.spi.function.OperatorType.NOT_EQUAL; -import static com.facebook.presto.spi.function.OperatorType.SATURATED_FLOOR_CAST; import static com.facebook.presto.spi.function.OperatorType.SUBTRACT; import static io.airlift.slice.Slices.utf8Slice; import static java.lang.Float.floatToRawIntBits; import static java.lang.Float.intBitsToFloat; import static java.lang.String.valueOf; -import static java.math.RoundingMode.FLOOR; public final class RealOperators { - private static final float MIN_LONG_AS_FLOAT = -0x1p63f; - private static final float MAX_LONG_PLUS_ONE_AS_FLOAT = 0x1p63f; - private static final float MIN_INTEGER_AS_FLOAT = -0x1p31f; - private static final float MAX_INTEGER_PLUS_ONE_AS_FLOAT = 0x1p31f; - private static final float MIN_SHORT_AS_FLOAT = -0x1p15f; - private static final float MAX_SHORT_PLUS_ONE_AS_FLOAT = 0x1p15f; - private static final float MIN_BYTE_AS_FLOAT = -0x1p7f; - private static final float MAX_BYTE_PLUS_ONE_AS_FLOAT = 0x1p7f; - private RealOperators() { } @@ -246,44 +234,4 @@ public static boolean isDistinctFrom( } return notEqual(left, right); } - - @ScalarOperator(SATURATED_FLOOR_CAST) - @SqlType(StandardTypes.BIGINT) - public static long saturatedFloorCastToBigint(@SqlType(StandardTypes.REAL) long value) - { - return saturatedFloorCastToLong(value, Long.MIN_VALUE, MIN_LONG_AS_FLOAT, Long.MAX_VALUE, MAX_LONG_PLUS_ONE_AS_FLOAT); - } - - @ScalarOperator(SATURATED_FLOOR_CAST) - @SqlType(StandardTypes.INTEGER) - public static long saturatedFloorCastToInteger(@SqlType(StandardTypes.REAL) long value) - { - return saturatedFloorCastToLong(value, Integer.MIN_VALUE, MIN_INTEGER_AS_FLOAT, Integer.MAX_VALUE, MAX_INTEGER_PLUS_ONE_AS_FLOAT); - } - - @ScalarOperator(SATURATED_FLOOR_CAST) - @SqlType(StandardTypes.SMALLINT) - public static long saturatedFloorCastToSmallint(@SqlType(StandardTypes.REAL) long value) - { - return saturatedFloorCastToLong(value, Short.MIN_VALUE, MIN_SHORT_AS_FLOAT, Short.MAX_VALUE, MAX_SHORT_PLUS_ONE_AS_FLOAT); - } - - @ScalarOperator(SATURATED_FLOOR_CAST) - @SqlType(StandardTypes.TINYINT) - public static long saturatedFloorCastToTinyint(@SqlType(StandardTypes.REAL) long value) - { - return saturatedFloorCastToLong(value, Byte.MIN_VALUE, MIN_BYTE_AS_FLOAT, Byte.MAX_VALUE, MAX_BYTE_PLUS_ONE_AS_FLOAT); - } - - private static long saturatedFloorCastToLong(long valueBits, long minValue, float minValueAsDouble, long maxValue, float maxValuePlusOneAsDouble) - { - float value = intBitsToFloat((int) valueBits); - if (value <= minValueAsDouble) { - return minValue; - } - if (value + 1 >= maxValuePlusOneAsDouble) { - return maxValue; - } - return DoubleMath.roundToLong(value, FLOOR); - } } diff --git a/presto-main/src/main/java/com/facebook/presto/type/SmallintOperators.java b/presto-main/src/main/java/com/facebook/presto/type/SmallintOperators.java index b904f15dd068..6a3107f2a9f2 100644 --- a/presto-main/src/main/java/com/facebook/presto/type/SmallintOperators.java +++ b/presto-main/src/main/java/com/facebook/presto/type/SmallintOperators.java @@ -41,7 +41,6 @@ import static com.facebook.presto.spi.function.OperatorType.MULTIPLY; import static com.facebook.presto.spi.function.OperatorType.NEGATION; import static com.facebook.presto.spi.function.OperatorType.NOT_EQUAL; -import static com.facebook.presto.spi.function.OperatorType.SATURATED_FLOOR_CAST; import static com.facebook.presto.spi.function.OperatorType.SUBTRACT; import static io.airlift.slice.Slices.utf8Slice; import static java.lang.Float.floatToRawIntBits; @@ -254,11 +253,4 @@ public static boolean isDistinctFrom( } return notEqual(left, right); } - - @ScalarOperator(SATURATED_FLOOR_CAST) - @SqlType(StandardTypes.TINYINT) - public static long saturatedFloorCastToTinyint(@SqlType(StandardTypes.SMALLINT) long value) - { - return SignedBytes.saturatedCast(value); - } } diff --git a/presto-main/src/main/java/com/facebook/presto/type/TypeRegistry.java b/presto-main/src/main/java/com/facebook/presto/type/TypeRegistry.java index 71a282731bfa..21c6edae412e 100644 --- a/presto-main/src/main/java/com/facebook/presto/type/TypeRegistry.java +++ b/presto-main/src/main/java/com/facebook/presto/type/TypeRegistry.java @@ -83,7 +83,7 @@ public final class TypeRegistry public TypeRegistry() { - this(ImmutableSet.of()); + this(ImmutableSet.of()); } @Inject @@ -272,7 +272,7 @@ public Optional getCommonSuperType(Type firstType, Type secondType) return Optional.empty(); } - private static Type getCommonSuperTypeForDecimal(DecimalType firstType, DecimalType secondType) + private Type getCommonSuperTypeForDecimal(DecimalType firstType, DecimalType secondType) { int targetScale = Math.max(firstType.getScale(), secondType.getScale()); int targetPrecision = Math.max(firstType.getPrecision() - firstType.getScale(), secondType.getPrecision() - secondType.getScale()) + targetScale; @@ -281,7 +281,7 @@ private static Type getCommonSuperTypeForDecimal(DecimalType firstType, DecimalT return createDecimalType(targetPrecision, targetScale); } - private static Type getCommonSuperTypeForVarchar(VarcharType firstType, VarcharType secondType) + private Type getCommonSuperTypeForVarchar(VarcharType firstType, VarcharType secondType) { return createVarcharType(Math.max(firstType.getLength(), secondType.getLength())); } @@ -322,7 +322,6 @@ public void addParametricType(ParametricType parametricType) * coerceTypeBase and isCovariantParametrizedType defines all hand-coded rules for type coercion. * Other methods should reference these two functions instead of hand-code new rules. */ - @Override public Optional coerceTypeBase(Type sourceType, String resultTypeBase) { String sourceTypeName = sourceType.getTypeSignature().getBase(); diff --git a/presto-main/src/main/java/com/facebook/presto/type/VarcharOperators.java b/presto-main/src/main/java/com/facebook/presto/type/VarcharOperators.java index 7ec09681a6ec..ff90e904bc77 100644 --- a/presto-main/src/main/java/com/facebook/presto/type/VarcharOperators.java +++ b/presto-main/src/main/java/com/facebook/presto/type/VarcharOperators.java @@ -41,58 +41,58 @@ private VarcharOperators() { } - @LiteralParameters("x") + @LiteralParameters({"x", "y"}) @ScalarOperator(EQUAL) @SqlType(StandardTypes.BOOLEAN) - public static boolean equal(@SqlType("varchar(x)") Slice left, @SqlType("varchar(x)") Slice right) + public static boolean equal(@SqlType("varchar(x)") Slice left, @SqlType("varchar(y)") Slice right) { return left.equals(right); } - @LiteralParameters("x") + @LiteralParameters({"x", "y"}) @ScalarOperator(NOT_EQUAL) @SqlType(StandardTypes.BOOLEAN) - public static boolean notEqual(@SqlType("varchar(x)") Slice left, @SqlType("varchar(x)") Slice right) + public static boolean notEqual(@SqlType("varchar(x)") Slice left, @SqlType("varchar(y)") Slice right) { return !left.equals(right); } - @LiteralParameters("x") + @LiteralParameters({"x", "y"}) @ScalarOperator(LESS_THAN) @SqlType(StandardTypes.BOOLEAN) - public static boolean lessThan(@SqlType("varchar(x)") Slice left, @SqlType("varchar(x)") Slice right) + public static boolean lessThan(@SqlType("varchar(x)") Slice left, @SqlType("varchar(y)") Slice right) { return left.compareTo(right) < 0; } - @LiteralParameters("x") + @LiteralParameters({"x", "y"}) @ScalarOperator(LESS_THAN_OR_EQUAL) @SqlType(StandardTypes.BOOLEAN) - public static boolean lessThanOrEqual(@SqlType("varchar(x)") Slice left, @SqlType("varchar(x)") Slice right) + public static boolean lessThanOrEqual(@SqlType("varchar(x)") Slice left, @SqlType("varchar(y)") Slice right) { return left.compareTo(right) <= 0; } - @LiteralParameters("x") + @LiteralParameters({"x", "y"}) @ScalarOperator(GREATER_THAN) @SqlType(StandardTypes.BOOLEAN) - public static boolean greaterThan(@SqlType("varchar(x)") Slice left, @SqlType("varchar(x)") Slice right) + public static boolean greaterThan(@SqlType("varchar(x)") Slice left, @SqlType("varchar(y)") Slice right) { return left.compareTo(right) > 0; } - @LiteralParameters("x") + @LiteralParameters({"x", "y"}) @ScalarOperator(GREATER_THAN_OR_EQUAL) @SqlType(StandardTypes.BOOLEAN) - public static boolean greaterThanOrEqual(@SqlType("varchar(x)") Slice left, @SqlType("varchar(x)") Slice right) + public static boolean greaterThanOrEqual(@SqlType("varchar(x)") Slice left, @SqlType("varchar(y)") Slice right) { return left.compareTo(right) >= 0; } - @LiteralParameters("x") + @LiteralParameters({"x", "y", "z"}) @ScalarOperator(BETWEEN) @SqlType(StandardTypes.BOOLEAN) - public static boolean between(@SqlType("varchar(x)") Slice value, @SqlType("varchar(x)") Slice min, @SqlType("varchar(x)") Slice max) + public static boolean between(@SqlType("varchar(x)") Slice value, @SqlType("varchar(y)") Slice min, @SqlType("varchar(z)") Slice max) { return min.compareTo(value) <= 0 && value.compareTo(max) <= 0; } diff --git a/presto-main/src/main/java/com/facebook/presto/type/VarcharParametricType.java b/presto-main/src/main/java/com/facebook/presto/type/VarcharParametricType.java index 11b386a348d1..9f07bcbb4975 100644 --- a/presto-main/src/main/java/com/facebook/presto/type/VarcharParametricType.java +++ b/presto-main/src/main/java/com/facebook/presto/type/VarcharParametricType.java @@ -21,8 +21,6 @@ import java.util.List; -import static com.facebook.presto.spi.type.VarcharType.createUnboundedVarcharType; - public class VarcharParametricType implements ParametricType { @@ -38,7 +36,7 @@ public String getName() public Type createType(List parameters) { if (parameters.isEmpty()) { - return createUnboundedVarcharType(); + return VarcharType.createVarcharType(VarcharType.MAX_LENGTH); } if (parameters.size() != 1) { throw new IllegalArgumentException("Expected at most one parameter for VARCHAR"); diff --git a/presto-main/src/test/java/com/facebook/presto/TestHiddenColumns.java b/presto-main/src/test/java/com/facebook/presto/TestHiddenColumns.java index ffa1d6334a86..986f6baf8c75 100644 --- a/presto-main/src/test/java/com/facebook/presto/TestHiddenColumns.java +++ b/presto-main/src/test/java/com/facebook/presto/TestHiddenColumns.java @@ -31,7 +31,7 @@ public class TestHiddenColumns public TestHiddenColumns() { runner = new LocalQueryRunner(TEST_SESSION); - runner.createCatalog(TEST_SESSION.getCatalog().get(), new TpchConnectorFactory(1), ImmutableMap.of()); + runner.createCatalog(TEST_SESSION.getCatalog().get(), new TpchConnectorFactory(1), ImmutableMap.of()); } @AfterClass @@ -46,10 +46,10 @@ public void destroy() public void testDescribeTable() throws Exception { - MaterializedResult expected = MaterializedResult.resultBuilder(TEST_SESSION, VARCHAR, VARCHAR, VARCHAR, VARCHAR) - .row("regionkey", "bigint", "", "") - .row("name", "varchar(25)", "", "") - .row("comment", "varchar(152)", "", "") + MaterializedResult expected = MaterializedResult.resultBuilder(TEST_SESSION, VARCHAR, VARCHAR, VARCHAR) + .row("regionkey", "bigint", "") + .row("name", "varchar(25)", "") + .row("comment", "varchar(152)", "") .build(); assertEquals(runner.execute("DESC REGION"), expected); } diff --git a/presto-main/src/test/java/com/facebook/presto/block/AbstractTestBlock.java b/presto-main/src/test/java/com/facebook/presto/block/AbstractTestBlock.java index 8700d8370d5f..4a893d11afe5 100644 --- a/presto-main/src/test/java/com/facebook/presto/block/AbstractTestBlock.java +++ b/presto-main/src/test/java/com/facebook/presto/block/AbstractTestBlock.java @@ -93,9 +93,6 @@ protected void assertBlockPosition(Block block, int position, T expectedValu assertPositionValue(block.getRegion(position, 1), 0, expectedValue); assertPositionValue(block.getRegion(0, position + 1), position, expectedValue); assertPositionValue(block.getRegion(position, block.getPositionCount() - position), 0, expectedValue); - assertPositionValue(copyBlock(block.getRegion(position, 1)), 0, expectedValue); - assertPositionValue(copyBlock(block.getRegion(0, position + 1)), position, expectedValue); - assertPositionValue(copyBlock(block.getRegion(position, block.getPositionCount() - position)), 0, expectedValue); assertPositionValue(block.copyRegion(position, 1), 0, expectedValue); assertPositionValue(block.copyRegion(0, position + 1), position, expectedValue); assertPositionValue(block.copyRegion(position, block.getPositionCount() - position), 0, expectedValue); diff --git a/presto-main/src/test/java/com/facebook/presto/block/TestArrayBlock.java b/presto-main/src/test/java/com/facebook/presto/block/TestArrayBlock.java index 0fcc480593aa..487039dc5967 100644 --- a/presto-main/src/test/java/com/facebook/presto/block/TestArrayBlock.java +++ b/presto-main/src/test/java/com/facebook/presto/block/TestArrayBlock.java @@ -103,7 +103,7 @@ public void testWithArrayBlock() assertBlockFilteredPositions(expectedValuesWithNull, blockBuilderWithNull.build(), Ints.asList(2, 3, 4, 9, 13, 14)); } - private static BlockBuilder createBlockBuilderWithValues(long[][][] expectedValues) + private BlockBuilder createBlockBuilderWithValues(long[][][] expectedValues) { BlockBuilder blockBuilder = new ArrayBlockBuilder(new ArrayBlockBuilder(BIGINT, new BlockBuilderStatus(), 100, 100), new BlockBuilderStatus(), 100); for (long[][] expectedValue : expectedValues) { @@ -130,7 +130,7 @@ private static BlockBuilder createBlockBuilderWithValues(long[][][] expectedValu return blockBuilder; } - private static BlockBuilder createBlockBuilderWithValues(long[][] expectedValues) + private BlockBuilder createBlockBuilderWithValues(long[][] expectedValues) { BlockBuilder blockBuilder = new ArrayBlockBuilder(BIGINT, new BlockBuilderStatus(), 100, 100); for (long[] expectedValue : expectedValues) { @@ -148,7 +148,7 @@ private static BlockBuilder createBlockBuilderWithValues(long[][] expectedValues return blockBuilder; } - private static BlockBuilder createBlockBuilderWithValues(Slice[][] expectedValues) + private BlockBuilder createBlockBuilderWithValues(Slice[][] expectedValues) { BlockBuilder blockBuilder = new ArrayBlockBuilder(VARCHAR, new BlockBuilderStatus(), 100, 100); for (Slice[] expectedValue : expectedValues) { diff --git a/presto-main/src/test/java/com/facebook/presto/block/TestPagesSerde.java b/presto-main/src/test/java/com/facebook/presto/block/TestPagesSerde.java index 90b85924b80a..620af7b6ec88 100644 --- a/presto-main/src/test/java/com/facebook/presto/block/TestPagesSerde.java +++ b/presto-main/src/test/java/com/facebook/presto/block/TestPagesSerde.java @@ -54,7 +54,7 @@ public void testRoundTrip() DynamicSliceOutput sliceOutput = new DynamicSliceOutput(1024); writePages(blockEncodingManager, sliceOutput, expectedPage, expectedPage, expectedPage); - List types = ImmutableList.of(VARCHAR, VARCHAR, VARCHAR); + List types = ImmutableList.of(VARCHAR, VARCHAR, VARCHAR); Iterator pageIterator = readPages(blockEncodingManager, sliceOutput.slice().getInput()); assertPageEquals(types, pageIterator.next(), expectedPage); assertPageEquals(types, pageIterator.next(), expectedPage); diff --git a/presto-main/src/test/java/com/facebook/presto/execution/MockRemoteTaskFactory.java b/presto-main/src/test/java/com/facebook/presto/execution/MockRemoteTaskFactory.java index c53bf7e9d0b8..8e3fbdbe45d0 100644 --- a/presto-main/src/test/java/com/facebook/presto/execution/MockRemoteTaskFactory.java +++ b/presto-main/src/test/java/com/facebook/presto/execution/MockRemoteTaskFactory.java @@ -28,6 +28,7 @@ import com.facebook.presto.spi.Node; import com.facebook.presto.spi.memory.MemoryPoolId; import com.facebook.presto.spi.predicate.TupleDomain; +import com.facebook.presto.spi.type.Type; import com.facebook.presto.sql.planner.Partitioning; import com.facebook.presto.sql.planner.PartitioningScheme; import com.facebook.presto.sql.planner.PlanFragment; @@ -103,7 +104,7 @@ public MockRemoteTask createTableScanTask(TaskId taskId, Node newNode, Listof(symbol, VARCHAR), SOURCE_DISTRIBUTION, ImmutableList.of(sourceId), new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), ImmutableList.of(symbol))); @@ -204,7 +205,7 @@ public TaskInfo getTaskInfo() return new TaskInfo(new TaskStatus(taskStateMachine.getTaskId(), TASK_INSTANCE_ID, nextTaskInfoVersion.getAndIncrement(), state, location, failures, 0, 0, new DataSize(0, BYTE)), DateTime.now(), outputBuffer.getInfo(), - ImmutableSet.of(), + ImmutableSet.of(), taskContext.getTaskStats(), true, false); diff --git a/presto-main/src/test/java/com/facebook/presto/execution/TaskTestUtils.java b/presto-main/src/test/java/com/facebook/presto/execution/TaskTestUtils.java index 78cde6529c18..f7c788c1764e 100644 --- a/presto-main/src/test/java/com/facebook/presto/execution/TaskTestUtils.java +++ b/presto-main/src/test/java/com/facebook/presto/execution/TaskTestUtils.java @@ -30,6 +30,7 @@ import com.facebook.presto.operator.index.IndexJoinLookupStats; import com.facebook.presto.spi.connector.ConnectorTransactionHandle; import com.facebook.presto.spi.predicate.TupleDomain; +import com.facebook.presto.spi.type.Type; import com.facebook.presto.spiller.BinarySpillerFactory; import com.facebook.presto.split.PageSinkManager; import com.facebook.presto.split.PageSourceManager; @@ -91,7 +92,7 @@ private TaskTestUtils() Optional.empty(), TupleDomain.all(), null), - ImmutableMap.of(SYMBOL, VARCHAR), + ImmutableMap.of(SYMBOL, VARCHAR), SOURCE_DISTRIBUTION, ImmutableList.of(TABLE_SCAN_NODE_ID), new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), ImmutableList.of(SYMBOL)) diff --git a/presto-main/src/test/java/com/facebook/presto/execution/TestSqlTask.java b/presto-main/src/test/java/com/facebook/presto/execution/TestSqlTask.java index de86e1479330..40ba81a8d0c2 100644 --- a/presto-main/src/test/java/com/facebook/presto/execution/TestSqlTask.java +++ b/presto-main/src/test/java/com/facebook/presto/execution/TestSqlTask.java @@ -15,6 +15,7 @@ import com.facebook.presto.OutputBuffers; import com.facebook.presto.OutputBuffers.OutputBufferId; +import com.facebook.presto.ScheduledSplit; import com.facebook.presto.TaskSource; import com.facebook.presto.client.NodeVersion; import com.facebook.presto.event.query.QueryMonitor; @@ -107,7 +108,7 @@ public void testEmptyQuery() TaskInfo taskInfo = sqlTask.updateTask(TEST_SESSION, Optional.of(PLAN_FRAGMENT), - ImmutableList.of(), + ImmutableList.of(), createInitialEmptyOutputBuffers(PARTITIONED) .withNoMoreBufferIds()); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.RUNNING); @@ -117,7 +118,7 @@ public void testEmptyQuery() taskInfo = sqlTask.updateTask(TEST_SESSION, Optional.of(PLAN_FRAGMENT), - ImmutableList.of(new TaskSource(TABLE_SCAN_NODE_ID, ImmutableSet.of(), true)), + ImmutableList.of(new TaskSource(TABLE_SCAN_NODE_ID, ImmutableSet.of(), true)), createInitialEmptyOutputBuffers(PARTITIONED) .withNoMoreBufferIds()); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.FINISHED); @@ -170,7 +171,7 @@ public void testCancel() TaskInfo taskInfo = sqlTask.updateTask(TEST_SESSION, Optional.of(PLAN_FRAGMENT), - ImmutableList.of(), + ImmutableList.of(), createInitialEmptyOutputBuffers(PARTITIONED) .withBuffer(OUT, 0) .withNoMoreBufferIds()); @@ -227,7 +228,7 @@ public void testBufferCloseOnFinish() assertFalse(bufferResult.isDone()); // close the sources (no splits will ever be added) - updateTask(sqlTask, ImmutableList.of(new TaskSource(TABLE_SCAN_NODE_ID, ImmutableSet.of(), true)), outputBuffers); + updateTask(sqlTask, ImmutableList.of(new TaskSource(TABLE_SCAN_NODE_ID, ImmutableSet.of(), true)), outputBuffers); // finish the task by calling abort on it sqlTask.abortTaskResults(OUT); @@ -300,7 +301,7 @@ public SqlTask createInitialTask() new QueryContext(new QueryId("query"), new DataSize(1, MEGABYTE), new MemoryPool(new MemoryPoolId("test"), new DataSize(1, GIGABYTE)), new MemoryPool(new MemoryPoolId("testSystem"), new DataSize(1, GIGABYTE)), taskNotificationExecutor), sqlTaskExecutionFactory, taskNotificationExecutor, - Functions.identity(), + Functions.identity(), new DataSize(32, MEGABYTE), true); } diff --git a/presto-main/src/test/java/com/facebook/presto/execution/TestSqlTaskManager.java b/presto-main/src/test/java/com/facebook/presto/execution/TestSqlTaskManager.java index 7a2f1add57bc..ae96f4d46a52 100644 --- a/presto-main/src/test/java/com/facebook/presto/execution/TestSqlTaskManager.java +++ b/presto-main/src/test/java/com/facebook/presto/execution/TestSqlTaskManager.java @@ -14,6 +14,7 @@ package com.facebook.presto.execution; import com.facebook.presto.OutputBuffers.OutputBufferId; +import com.facebook.presto.ScheduledSplit; import com.facebook.presto.TaskSource; import com.facebook.presto.client.NodeVersion; import com.facebook.presto.event.query.QueryMonitor; @@ -86,7 +87,7 @@ public void testEmptyQuery() TaskInfo taskInfo = sqlTaskManager.updateTask(TEST_SESSION, taskId, Optional.of(PLAN_FRAGMENT), - ImmutableList.of(), + ImmutableList.of(), createInitialEmptyOutputBuffers(PARTITIONED) .withNoMoreBufferIds()); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.RUNNING); @@ -97,7 +98,7 @@ public void testEmptyQuery() taskInfo = sqlTaskManager.updateTask(TEST_SESSION, taskId, Optional.of(PLAN_FRAGMENT), - ImmutableList.of(new TaskSource(TABLE_SCAN_NODE_ID, ImmutableSet.of(), true)), + ImmutableList.of(new TaskSource(TABLE_SCAN_NODE_ID, ImmutableSet.of(), true)), createInitialEmptyOutputBuffers(PARTITIONED) .withNoMoreBufferIds()); assertEquals(taskInfo.getTaskStatus().getState(), TaskState.FINISHED); @@ -156,7 +157,7 @@ public void testCancel() TaskInfo taskInfo = sqlTaskManager.updateTask(TEST_SESSION, taskId, Optional.of(PLAN_FRAGMENT), - ImmutableList.of(), + ImmutableList.of(), createInitialEmptyOutputBuffers(PARTITIONED) .withBuffer(OUT, 0) .withNoMoreBufferIds()); @@ -186,7 +187,7 @@ public void testAbort() TaskInfo taskInfo = sqlTaskManager.updateTask(TEST_SESSION, taskId, Optional.of(PLAN_FRAGMENT), - ImmutableList.of(), + ImmutableList.of(), createInitialEmptyOutputBuffers(PARTITIONED) .withBuffer(OUT, 0) .withNoMoreBufferIds()); @@ -245,7 +246,7 @@ public void testRemoveOldTasks() TaskInfo taskInfo = sqlTaskManager.updateTask(TEST_SESSION, taskId, Optional.of(PLAN_FRAGMENT), - ImmutableList.of(), + ImmutableList.of(), createInitialEmptyOutputBuffers(PARTITIONED) .withBuffer(OUT, 0) .withNoMoreBufferIds()); diff --git a/presto-main/src/test/java/com/facebook/presto/execution/TestStageStateMachine.java b/presto-main/src/test/java/com/facebook/presto/execution/TestStageStateMachine.java index 003e31679eb0..c1a5a629a653 100644 --- a/presto-main/src/test/java/com/facebook/presto/execution/TestStageStateMachine.java +++ b/presto-main/src/test/java/com/facebook/presto/execution/TestStageStateMachine.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.execution; +import com.facebook.presto.spi.type.Type; import com.facebook.presto.sql.planner.Partitioning; import com.facebook.presto.sql.planner.PartitioningScheme; import com.facebook.presto.sql.planner.PlanFragment; @@ -324,7 +325,7 @@ private static PlanFragment createValuesPlan() new ValuesNode(valuesNodeId, ImmutableList.of(symbol), ImmutableList.of(ImmutableList.of(new StringLiteral("foo")))), - ImmutableMap.of(symbol, VARCHAR), + ImmutableMap.of(symbol, VARCHAR), SOURCE_DISTRIBUTION, ImmutableList.of(valuesNodeId), new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), ImmutableList.of(symbol))); diff --git a/presto-main/src/test/java/com/facebook/presto/execution/resourceGroups/TestResourceGroups.java b/presto-main/src/test/java/com/facebook/presto/execution/resourceGroups/TestResourceGroups.java index c74bdbccb24c..28c7c4b1e358 100644 --- a/presto-main/src/test/java/com/facebook/presto/execution/resourceGroups/TestResourceGroups.java +++ b/presto-main/src/test/java/com/facebook/presto/execution/resourceGroups/TestResourceGroups.java @@ -62,7 +62,7 @@ public void testQueueFull() MockQueryExecution query3 = new MockQueryExecution(0); root.run(query3); assertEquals(query3.getState(), FAILED); - assertEquals(query3.getFailureCause().getMessage(), "Too many queued queries for \"root\""); + assertEquals(query3.getFailureCause().getMessage(), "Too many queued queries for \"root\"!"); } @Test(timeOut = 10_000) diff --git a/presto-main/src/test/java/com/facebook/presto/execution/scheduler/TestPhasedExecutionSchedule.java b/presto-main/src/test/java/com/facebook/presto/execution/scheduler/TestPhasedExecutionSchedule.java index 4a4ccf48cdbc..1f5346eeccd4 100644 --- a/presto-main/src/test/java/com/facebook/presto/execution/scheduler/TestPhasedExecutionSchedule.java +++ b/presto-main/src/test/java/com/facebook/presto/execution/scheduler/TestPhasedExecutionSchedule.java @@ -30,6 +30,7 @@ import com.facebook.presto.sql.planner.plan.RemoteSourceNode; import com.facebook.presto.sql.planner.plan.TableScanNode; import com.facebook.presto.sql.planner.plan.UnionNode; +import com.facebook.presto.sql.tree.Expression; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableListMultimap; import com.google.common.collect.ImmutableMap; @@ -194,9 +195,9 @@ private static PlanFragment createBroadcastJoinPlanFragment(String name, PlanFra tableScan, new RemoteSourceNode(new PlanNodeId("build_id"), buildFragment.getId(), ImmutableList.of()), ImmutableList.of(), - Optional.empty(), - Optional.empty(), - Optional.empty()); + Optional.empty(), + Optional.empty(), + Optional.empty()); return createFragment(join); } @@ -210,8 +211,8 @@ private static PlanFragment createJoinPlanFragment(JoinNode.Type joinType, Strin new RemoteSourceNode(new PlanNodeId("build_id"), buildFragment.getId(), ImmutableList.of()), ImmutableList.of(), Optional.empty(), - Optional.empty(), - Optional.empty()); + Optional.empty(), + Optional.empty()); return createFragment(planNode); } diff --git a/presto-main/src/test/java/com/facebook/presto/execution/scheduler/TestSourcePartitionedScheduler.java b/presto-main/src/test/java/com/facebook/presto/execution/scheduler/TestSourcePartitionedScheduler.java index 88be28dfbb35..64445e1e355a 100644 --- a/presto-main/src/test/java/com/facebook/presto/execution/scheduler/TestSourcePartitionedScheduler.java +++ b/presto-main/src/test/java/com/facebook/presto/execution/scheduler/TestSourcePartitionedScheduler.java @@ -35,6 +35,7 @@ import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.QueryId; import com.facebook.presto.spi.predicate.TupleDomain; +import com.facebook.presto.spi.type.Type; import com.facebook.presto.split.ConnectorAwareSplitSource; import com.facebook.presto.split.SplitSource; import com.facebook.presto.sql.planner.Partitioning; @@ -440,9 +441,9 @@ private static StageExecutionPlan createPlan(ConnectorSplitSource splitSource) new RemoteSourceNode(new PlanNodeId("remote_id"), new PlanFragmentId("plan_fragment_id"), ImmutableList.of()), ImmutableList.of(), Optional.empty(), - Optional.empty(), - Optional.empty()), - ImmutableMap.of(symbol, VARCHAR), + Optional.empty(), + Optional.empty()), + ImmutableMap.of(symbol, VARCHAR), SOURCE_DISTRIBUTION, ImmutableList.of(tableScanNodeId), new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), ImmutableList.of(symbol))); diff --git a/presto-main/src/test/java/com/facebook/presto/failureDetector/TestHeartbeatFailureDetector.java b/presto-main/src/test/java/com/facebook/presto/failureDetector/TestHeartbeatFailureDetector.java index 41147d9f9966..aee6e94227e9 100644 --- a/presto-main/src/test/java/com/facebook/presto/failureDetector/TestHeartbeatFailureDetector.java +++ b/presto-main/src/test/java/com/facebook/presto/failureDetector/TestHeartbeatFailureDetector.java @@ -90,7 +90,7 @@ public void configure(Binder binder) public static class FooResource { @GET - public static String hello() + public String hello() { return "hello"; } diff --git a/presto-main/src/test/java/com/facebook/presto/memory/TestMemoryPools.java b/presto-main/src/test/java/com/facebook/presto/memory/TestMemoryPools.java index 239c899a9990..a26a1c009c3a 100644 --- a/presto-main/src/test/java/com/facebook/presto/memory/TestMemoryPools.java +++ b/presto-main/src/test/java/com/facebook/presto/memory/TestMemoryPools.java @@ -54,7 +54,7 @@ public void testBlocking() LocalQueryRunner localQueryRunner = queryRunnerWithInitialTransaction(session); // add tpch - localQueryRunner.createCatalog("tpch", new TpchConnectorFactory(1), ImmutableMap.of()); + localQueryRunner.createCatalog("tpch", new TpchConnectorFactory(1), ImmutableMap.of()); // reserve all the memory in the pool MemoryPool pool = new MemoryPool(new MemoryPoolId("test"), new DataSize(10, MEGABYTE)); diff --git a/presto-main/src/test/java/com/facebook/presto/metadata/TestFunctionRegistry.java b/presto-main/src/test/java/com/facebook/presto/metadata/TestFunctionRegistry.java index 87715e255956..5a403c8892b7 100644 --- a/presto-main/src/test/java/com/facebook/presto/metadata/TestFunctionRegistry.java +++ b/presto-main/src/test/java/com/facebook/presto/metadata/TestFunctionRegistry.java @@ -43,7 +43,6 @@ import static com.facebook.presto.spi.type.HyperLogLogType.HYPER_LOG_LOG; import static com.facebook.presto.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature; -import static com.facebook.presto.sql.analyzer.TypeSignatureProvider.fromTypeSignatures; import static com.facebook.presto.type.TypeUtils.resolveTypes; import static com.facebook.presto.util.ImmutableCollectors.toImmutableList; import static com.google.common.collect.Lists.transform; @@ -101,7 +100,7 @@ public void testMagicLiteralFunction() TypeRegistry typeManager = new TypeRegistry(); FunctionRegistry registry = new FunctionRegistry(typeManager, new BlockEncodingManager(typeManager), new FeaturesConfig()); - Signature function = registry.resolveFunction(QualifiedName.of(signature.getName()), fromTypeSignatures(signature.getArgumentTypes())); + Signature function = registry.resolveFunction(QualifiedName.of(signature.getName()), signature.getArgumentTypes()); assertEquals(function.getArgumentTypes(), ImmutableList.of(parseTypeSignature(StandardTypes.BIGINT))); assertEquals(signature.getReturnType().getBase(), StandardTypes.TIMESTAMP_WITH_TIME_ZONE); } @@ -317,12 +316,12 @@ private SignatureBuilder functionSignature(String... argumentTypes) return functionSignature(ImmutableList.copyOf(argumentTypes), "boolean"); } - private static SignatureBuilder functionSignature(List arguments, String returnType) + private SignatureBuilder functionSignature(List arguments, String returnType) { return functionSignature(arguments, returnType, ImmutableList.of()); } - private static SignatureBuilder functionSignature(List arguments, String returnType, List typeVariableConstraints) + private SignatureBuilder functionSignature(List arguments, String returnType, List typeVariableConstraints) { ImmutableSet literalParameters = ImmutableSet.of("p", "s", "p1", "s1", "p2", "s2", "p3", "s3"); List argumentSignatures = arguments.stream() @@ -391,7 +390,7 @@ private Signature resolveSignature() { FunctionRegistry functionRegistry = new FunctionRegistry(typeRegistry, blockEncoding, new FeaturesConfig()); functionRegistry.addFunctions(createFunctionsFromSignatures()); - return functionRegistry.resolveFunction(QualifiedName.of(TEST_FUNCTION_NAME), fromTypeSignatures(parameterTypes)); + return functionRegistry.resolveFunction(QualifiedName.of(TEST_FUNCTION_NAME), parameterTypes); } private List createFunctionsFromSignatures() @@ -433,7 +432,7 @@ public String getDescription() return functions.build(); } - private static List parseTypeSignatures(String... signatures) + private List parseTypeSignatures(String... signatures) { return ImmutableList.copyOf(signatures) .stream() diff --git a/presto-main/src/test/java/com/facebook/presto/metadata/TestInformationSchemaTableHandle.java b/presto-main/src/test/java/com/facebook/presto/metadata/TestInformationSchemaTableHandle.java index 8b61ee065868..4e56a9be994d 100644 --- a/presto-main/src/test/java/com/facebook/presto/metadata/TestInformationSchemaTableHandle.java +++ b/presto-main/src/test/java/com/facebook/presto/metadata/TestInformationSchemaTableHandle.java @@ -33,7 +33,7 @@ @Test(singleThreaded = true) public class TestInformationSchemaTableHandle { - private static final Map SCHEMA_AS_MAP = ImmutableMap.of( + private static final Map SCHEMA_AS_MAP = ImmutableMap.of( "@type", "$info_schema", "catalogName", "information_schema_catalog", "schemaName", "information_schema_schema", diff --git a/presto-main/src/test/java/com/facebook/presto/metadata/TestSignature.java b/presto-main/src/test/java/com/facebook/presto/metadata/TestSignature.java index 847ad5f1b1ca..0b9cc473a953 100644 --- a/presto-main/src/test/java/com/facebook/presto/metadata/TestSignature.java +++ b/presto-main/src/test/java/com/facebook/presto/metadata/TestSignature.java @@ -17,6 +17,7 @@ import com.facebook.presto.spi.type.Type; import com.facebook.presto.type.TypeDeserializer; import com.facebook.presto.type.TypeRegistry; +import com.fasterxml.jackson.databind.JsonDeserializer; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.airlift.json.JsonCodec; @@ -34,7 +35,7 @@ public class TestSignature public void testSerializationRoundTrip() { ObjectMapperProvider objectMapperProvider = new ObjectMapperProvider(); - objectMapperProvider.setJsonDeserializers(ImmutableMap.of(Type.class, new TypeDeserializer(new TypeRegistry()))); + objectMapperProvider.setJsonDeserializers(ImmutableMap., JsonDeserializer>of(Type.class, new TypeDeserializer(new TypeRegistry()))); JsonCodec codec = new JsonCodecFactory(objectMapperProvider, true).jsonCodec(Signature.class); Signature expected = new Signature( diff --git a/presto-main/src/test/java/com/facebook/presto/metadata/TestSignatureBinder.java b/presto-main/src/test/java/com/facebook/presto/metadata/TestSignatureBinder.java index 8ea25b21fa98..2a54c8cbdebc 100644 --- a/presto-main/src/test/java/com/facebook/presto/metadata/TestSignatureBinder.java +++ b/presto-main/src/test/java/com/facebook/presto/metadata/TestSignatureBinder.java @@ -15,10 +15,10 @@ import com.facebook.presto.spi.type.BigintType; import com.facebook.presto.spi.type.DecimalType; +import com.facebook.presto.spi.type.DoubleType; import com.facebook.presto.spi.type.StandardTypes; import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.TypeSignature; -import com.facebook.presto.sql.analyzer.TypeSignatureProvider; import com.facebook.presto.type.TypeRegistry; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -34,12 +34,7 @@ import static com.facebook.presto.metadata.Signature.typeVariable; import static com.facebook.presto.metadata.Signature.withVariadicBound; import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; -import static com.facebook.presto.spi.type.DoubleType.DOUBLE; -import static com.facebook.presto.spi.type.IntegerType.INTEGER; import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature; -import static com.facebook.presto.spi.type.VarcharType.VARCHAR; -import static com.facebook.presto.sql.analyzer.TypeSignatureProvider.fromTypes; -import static java.lang.String.format; import static java.util.Objects.requireNonNull; import static java.util.stream.Collectors.toList; import static org.testng.Assert.assertEquals; @@ -987,98 +982,12 @@ public void testUnknownCoercion() .fails(); } - @Test - public void testFunction() - throws Exception - { - Signature simple = functionSignature() - .returnType(parseTypeSignature("boolean")) - .argumentTypes(parseTypeSignature("function(integer,integer)")) - .build(); - - assertThat(simple) - .boundTo("function(integer,integer)") - .succeeds(); - // TODO: This should eventually be supported - assertThat(simple) - .boundTo("function(integer,smallint)") - .withCoercion() - .fails(); - assertThat(simple) - .boundTo("function(integer,bigint)") - .withCoercion() - .fails(); - - Signature applyTwice = functionSignature() - .returnType(parseTypeSignature("V")) - .argumentTypes(parseTypeSignature("T"), parseTypeSignature("function(T,U)"), parseTypeSignature("function(U,V)")) - .typeVariableConstraints(typeVariable("T"), typeVariable("U"), typeVariable("V")) - .build(); - assertThat(applyTwice) - .boundTo("integer", "function(integer,varchar)", "function(varchar,double)") - .produces(BoundVariables.builder() - .setTypeVariable("T", INTEGER) - .setTypeVariable("U", VARCHAR) - .setTypeVariable("V", DOUBLE) - .build()); - assertThat(applyTwice) - .boundTo( - "integer", - new TypeSignatureProvider(functionArgumentTypes -> TypeSignature.parseTypeSignature("function(integer,varchar)")), - new TypeSignatureProvider(functionArgumentTypes -> TypeSignature.parseTypeSignature("function(varchar,double)"))) - .produces(BoundVariables.builder() - .setTypeVariable("T", INTEGER) - .setTypeVariable("U", VARCHAR) - .setTypeVariable("V", DOUBLE) - .build()); - assertThat(applyTwice) - .boundTo( - // pass function argument to non-function position of a function - new TypeSignatureProvider(functionArgumentTypes -> TypeSignature.parseTypeSignature("function(integer,varchar)")), - new TypeSignatureProvider(functionArgumentTypes -> TypeSignature.parseTypeSignature("function(integer,varchar)")), - new TypeSignatureProvider(functionArgumentTypes -> TypeSignature.parseTypeSignature("function(varchar,double)"))) - .fails(); - assertThat(applyTwice) - .boundTo( - new TypeSignatureProvider(functionArgumentTypes -> TypeSignature.parseTypeSignature("function(integer,varchar)")), - // pass non-function argument to function position of a function - "integer", - new TypeSignatureProvider(functionArgumentTypes -> TypeSignature.parseTypeSignature("function(varchar,double)"))) - .fails(); - - Signature flatMap = functionSignature() - .returnType(parseTypeSignature("array(T)")) - .argumentTypes(parseTypeSignature("array(T)"), parseTypeSignature("function(T, array(T))")) - .typeVariableConstraints(typeVariable("T")) - .build(); - assertThat(flatMap) - .boundTo("array(integer)", "function(integer, array(integer))") - .produces(BoundVariables.builder() - .setTypeVariable("T", INTEGER) - .build()); - - Signature varargApply = functionSignature() - .returnType(parseTypeSignature("T")) - .argumentTypes(parseTypeSignature("T"), parseTypeSignature("function(T, T)")) - .typeVariableConstraints(typeVariable("T")) - .setVariableArity(true) - .build(); - assertThat(varargApply) - .boundTo("integer", "function(integer, integer)", "function(integer, integer)", "function(integer, integer)") - .produces(BoundVariables.builder() - .setTypeVariable("T", INTEGER) - .build()); - assertThat(varargApply) - .boundTo("integer", "function(integer, integer)", "function(integer, double)", "function(double, double)") - .fails(); - } - @Test public void testBindParameters() throws Exception { BoundVariables boundVariables = BoundVariables.builder() - .setTypeVariable("T1", DOUBLE) + .setTypeVariable("T1", DoubleType.DOUBLE) .setTypeVariable("T2", BigintType.BIGINT) .setTypeVariable("T3", DecimalType.createDecimalType(5, 3)) .setLongVariable("p", 1L) @@ -1102,7 +1011,7 @@ public void testBindParameters() assertBindVariablesFails("T1(bigint)", boundVariables, "Unbounded parameters can not have parameters"); } - private static void assertBindVariablesFails(String typeSignature, BoundVariables boundVariables, String reason) + private void assertBindVariablesFails(String typeSignature, BoundVariables boundVariables, String reason) { try { SignatureBinder.applyBoundVariables(parseTypeSignature(typeSignature, ImmutableSet.of("p", "s")), boundVariables); @@ -1113,7 +1022,7 @@ private static void assertBindVariablesFails(String typeSignature, BoundVariable } } - private static void assertThat(String typeSignature, BoundVariables boundVariables, String expectedTypeSignature) + private void assertThat(String typeSignature, BoundVariables boundVariables, String expectedTypeSignature) { assertEquals( SignatureBinder.applyBoundVariables(parseTypeSignature(typeSignature, ImmutableSet.of("p", "s")), boundVariables).toString(), @@ -1121,7 +1030,7 @@ private static void assertThat(String typeSignature, BoundVariables boundVariabl ); } - private static SignatureBuilder functionSignature() + private SignatureBuilder functionSignature() { return new SignatureBuilder() .name("function") @@ -1150,7 +1059,7 @@ private BindSignatureAssertion assertThat(Signature function) private class BindSignatureAssertion { private final Signature function; - private List argumentTypes = null; + private List argumentTypes = null; private Type returnType = null; private boolean allowCoercion = false; @@ -1167,31 +1076,13 @@ public BindSignatureAssertion withCoercion() public BindSignatureAssertion boundTo(String... arguments) { - this.argumentTypes = fromTypes(types(arguments)); - return this; - } - - public BindSignatureAssertion boundTo(Object... arguments) - { - ImmutableList.Builder builder = ImmutableList.builder(); - for (Object argument : arguments) { - if (argument instanceof String) { - builder.add(new TypeSignatureProvider(TypeSignature.parseTypeSignature((String) argument))); - continue; - } - if (argument instanceof TypeSignatureProvider) { - builder.add((TypeSignatureProvider) argument); - continue; - } - throw new IllegalArgumentException(format("argument is of type %s. It should be String or TypeSignatureProvider", argument.getClass())); - } - this.argumentTypes = builder.build(); + this.argumentTypes = types(arguments); return this; } public BindSignatureAssertion boundTo(List arguments, String returnType) { - this.argumentTypes = fromTypes(types(arguments.toArray(new String[arguments.size()]))); + this.argumentTypes = types(arguments.toArray(new String[arguments.size()])); this.returnType = type(returnType); return this; } diff --git a/presto-main/src/test/java/com/facebook/presto/operator/BenchmarkGroupByHash.java b/presto-main/src/test/java/com/facebook/presto/operator/BenchmarkGroupByHash.java index 9f470be860f3..062bd05a5886 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/BenchmarkGroupByHash.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/BenchmarkGroupByHash.java @@ -261,7 +261,7 @@ public static class SingleChannelBenchmarkData public void setup() { pages = createPages(POSITIONS, GROUP_COUNT, ImmutableList.of(BIGINT), hashEnabled); - types = Collections.nCopies(1, BIGINT); + types = Collections.nCopies(1, BIGINT); channels = new int[1]; for (int i = 0; i < 1; i++) { channels[i] = i; @@ -306,9 +306,9 @@ public static class BenchmarkData @Setup public void setup() { - pages = createPages(POSITIONS, groupCount, Collections.nCopies(channelCount, BIGINT), hashEnabled); + pages = createPages(POSITIONS, groupCount, Collections.nCopies(channelCount, BIGINT), hashEnabled); hashChannel = hashEnabled ? Optional.of(channelCount) : Optional.empty(); - types = Collections.nCopies(channelCount, BIGINT); + types = Collections.nCopies(channelCount, BIGINT); channels = new int[channelCount]; for (int i = 0; i < channelCount; i++) { channels[i] = i; diff --git a/presto-main/src/test/java/com/facebook/presto/operator/BenchmarkHashBuildAndJoinOperators.java b/presto-main/src/test/java/com/facebook/presto/operator/BenchmarkHashBuildAndJoinOperators.java index 5762fbd9cae4..7589dc5718bd 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/BenchmarkHashBuildAndJoinOperators.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/BenchmarkHashBuildAndJoinOperators.java @@ -174,7 +174,6 @@ public static class JoinContext protected List probePages; - @Override @Setup public void setup() { diff --git a/presto-main/src/test/java/com/facebook/presto/operator/MockExchangeRequestProcessor.java b/presto-main/src/test/java/com/facebook/presto/operator/MockExchangeRequestProcessor.java index cce9b9a9c819..775c22cc0a7b 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/MockExchangeRequestProcessor.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/MockExchangeRequestProcessor.java @@ -85,7 +85,7 @@ public Response handle(Request request) throws Exception { if (request.getMethod().equalsIgnoreCase("DELETE")) { - return new TestingResponse(HttpStatus.NO_CONTENT, ImmutableListMultimap.of(), new byte[0]); + return new TestingResponse(HttpStatus.NO_CONTENT, ImmutableListMultimap.of(), new byte[0]); } // verify we got a data size and it parses correctly diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestDriver.java b/presto-main/src/test/java/com/facebook/presto/operator/TestDriver.java index ace037a134f4..d00f14c059db 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestDriver.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestDriver.java @@ -90,7 +90,7 @@ public void tearDown() @Test public void testNormalFinish() { - List types = ImmutableList.of(VARCHAR, BIGINT, BIGINT); + List types = ImmutableList.of(VARCHAR, BIGINT, BIGINT); ValuesOperator source = new ValuesOperator(driverContext.addOperatorContext(0, new PlanNodeId("test"), "values"), types, rowPagesBuilder(types) .addSequencePage(10, 20, 30, 40) .build()); @@ -112,7 +112,7 @@ public void testNormalFinish() @Test public void testAbruptFinish() { - List types = ImmutableList.of(VARCHAR, BIGINT, BIGINT); + List types = ImmutableList.of(VARCHAR, BIGINT, BIGINT); ValuesOperator source = new ValuesOperator(driverContext.addOperatorContext(0, new PlanNodeId("test"), "values"), types, rowPagesBuilder(types) .addSequencePage(10, 20, 30, 40) .build()); @@ -138,7 +138,7 @@ public void testAbruptFinish() public void testAddSourceFinish() { PlanNodeId sourceId = new PlanNodeId("source"); - final List types = ImmutableList.of(VARCHAR, BIGINT, BIGINT); + final List types = ImmutableList.of(VARCHAR, BIGINT, BIGINT); TableScanOperator source = new TableScanOperator(driverContext.addOperatorContext(99, new PlanNodeId("test"), "values"), sourceId, new PageSourceProvider() @@ -152,7 +152,7 @@ public ConnectorPageSource createPageSource(Session session, Split split, Listof()); PageConsumerOperator sink = createSinkOperator(source); Driver driver = new Driver(driverContext, source, sink); @@ -241,7 +241,7 @@ public void testBrokenOperatorAddSource() throws Exception { PlanNodeId sourceId = new PlanNodeId("source"); - final List types = ImmutableList.of(VARCHAR, BIGINT, BIGINT); + final List types = ImmutableList.of(VARCHAR, BIGINT, BIGINT); // create a table scan operator that does not block, which will cause the driver loop to busy wait TableScanOperator source = new NotBlockedTableScanOperator(driverContext.addOperatorContext(99, new PlanNodeId("test"), "values"), sourceId, @@ -256,7 +256,7 @@ public ConnectorPageSource createPageSource(Session session, Split split, Listof()); BrokenOperator brokenOperator = new BrokenOperator(driverContext.addOperatorContext(0, new PlanNodeId("test"), "source")); final Driver driver = new Driver(driverContext, source, brokenOperator); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestExchangeOperator.java b/presto-main/src/test/java/com/facebook/presto/operator/TestExchangeOperator.java index b6c73c1ae519..15afcb497a57 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestExchangeOperator.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestExchangeOperator.java @@ -74,7 +74,7 @@ @Test(singleThreaded = true) public class TestExchangeOperator { - private static final List TYPES = ImmutableList.of(VARCHAR); + private static final List TYPES = ImmutableList.of(VARCHAR); private static final Page PAGE = createSequencePage(TYPES, 10, 100); private static final BlockEncodingManager blockEncodingSerde = new BlockEncodingManager(new TypeRegistry()); @@ -156,7 +156,7 @@ public void testSimple() waitForFinished(operator); } - private static Split newRemoteSplit(String taskId) + private Split newRemoteSplit(String taskId) { return new Split(REMOTE_CONNECTOR_ID, new RemoteTransactionHandle(), new RemoteSplit(URI.create("http://localhost/" + taskId))); } @@ -274,7 +274,7 @@ private SourceOperator createExchangeOperator() return operator; } - private static List waitForPages(Operator operator, int expectedPageCount) + private List waitForPages(Operator operator, int expectedPageCount) throws InterruptedException { // read expected pages or until 10 seconds has passed @@ -330,7 +330,7 @@ private static List waitForPages(Operator operator, int expectedPageCount) return outputPages; } - private static void waitForFinished(Operator operator) + private void waitForFinished(Operator operator) throws InterruptedException { // wait for finished or until 10 seconds has passed diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestFilterAndProjectOperator.java b/presto-main/src/test/java/com/facebook/presto/operator/TestFilterAndProjectOperator.java index b99422909708..ae1f37b659f3 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestFilterAndProjectOperator.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestFilterAndProjectOperator.java @@ -97,7 +97,7 @@ public Set getInputChannels() 0, new PlanNodeId("test"), () -> new GenericPageProcessor(filter, ImmutableList.of(singleColumn(VARCHAR, 0), new Add5Projection(1))), - ImmutableList.of(VARCHAR, BIGINT)); + ImmutableList.of(VARCHAR, BIGINT)); MaterializedResult expected = MaterializedResult.resultBuilder(driverContext.getSession(), VARCHAR, BIGINT) .row("10", 15L) diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestHashAggregationOperator.java b/presto-main/src/test/java/com/facebook/presto/operator/TestHashAggregationOperator.java index 215332af68e8..e8ccafee5597 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestHashAggregationOperator.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestHashAggregationOperator.java @@ -25,15 +25,11 @@ import com.facebook.presto.spi.block.PageBuilderStatus; import com.facebook.presto.spi.type.StandardTypes; import com.facebook.presto.spi.type.Type; -import com.facebook.presto.spiller.Spiller; -import com.facebook.presto.spiller.SpillerFactory; -import com.facebook.presto.spiller.SpillerFactoryWithStats; import com.facebook.presto.sql.planner.plan.AggregationNode.Step; import com.facebook.presto.sql.planner.plan.PlanNodeId; import com.facebook.presto.testing.MaterializedResult; import com.google.common.collect.ImmutableList; import com.google.common.primitives.Ints; -import com.google.common.util.concurrent.SettableFuture; import io.airlift.slice.Slices; import io.airlift.units.DataSize; import io.airlift.units.DataSize.Unit; @@ -42,12 +38,10 @@ import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Optional; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import static com.facebook.presto.RowPagesBuilder.rowPagesBuilder; @@ -66,13 +60,11 @@ import static com.facebook.presto.spi.type.VarcharType.VARCHAR; import static com.facebook.presto.testing.MaterializedResult.resultBuilder; import static com.facebook.presto.testing.TestingTaskContext.createTaskContext; -import static io.airlift.concurrent.MoreFutures.toCompletableFuture; import static io.airlift.concurrent.Threads.daemonThreadsNamed; import static io.airlift.slice.SizeOf.SIZE_OF_DOUBLE; import static io.airlift.slice.SizeOf.SIZE_OF_LONG; import static io.airlift.testing.Assertions.assertEqualsIgnoreOrder; import static io.airlift.units.DataSize.Unit.MEGABYTE; -import static io.airlift.units.DataSize.succinctBytes; import static java.util.concurrent.Executors.newCachedThreadPool; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; @@ -91,7 +83,6 @@ public class TestHashAggregationOperator private ExecutorService executor; private DriverContext driverContext; - private SpillerFactory spillerFactory = new DummySpillerFactory(); @BeforeMethod public void setUp() @@ -103,20 +94,10 @@ public void setUp() .addDriverContext(); } - @DataProvider(name = "hashEnabled") - public static Object[][] hashEnabled() + @DataProvider(name = "hashEnabledValues") + public static Object[][] hashEnabledValuesProvider() { - return new Object[][] {{true}, {false}}; - } - - @DataProvider(name = "hashEnabledAndMemoryLimitBeforeSpillValues") - public static Object[][] hashEnabledAndMemoryLimitBeforeSpillValuesProvider() - { - return new Object[][] { - {true, 8, Integer.MAX_VALUE}, - {false, 0, 0}, - {false, 8, 0}, - {false, 8, Integer.MAX_VALUE}}; + return new Object[][] { { true }, { false } }; } @AfterMethod @@ -125,8 +106,8 @@ public void tearDown() executor.shutdownNow(); } - @Test(dataProvider = "hashEnabledAndMemoryLimitBeforeSpillValues") - public void testHashAggregation(boolean hashEnabled, long memoryLimitBeforeSpill, long memoryLimitForMergeWithMemory) + @Test(dataProvider = "hashEnabledValues") + public void testHashAggregation(boolean hashEnabled) throws Exception { MetadataManager metadata = MetadataManager.createTestMetadataManager(); @@ -160,11 +141,7 @@ public void testHashAggregation(boolean hashEnabled, long memoryLimitBeforeSpill rowPagesBuilder.getHashChannel(), Optional.empty(), 100_000, - new DataSize(16, MEGABYTE), - memoryLimitBeforeSpill > 0, - succinctBytes(memoryLimitBeforeSpill), - succinctBytes(memoryLimitForMergeWithMemory), - spillerFactory); + new DataSize(16, MEGABYTE)); MaterializedResult expected = resultBuilder(driverContext.getSession(), VARCHAR, BIGINT, BIGINT, DOUBLE, VARCHAR, BIGINT, BIGINT) .row("0", 3L, 0L, 0.0, "300", 3L, 3L) @@ -182,7 +159,7 @@ public void testHashAggregation(boolean hashEnabled, long memoryLimitBeforeSpill assertOperatorEqualsIgnoreOrder(operatorFactory, driverContext, input, expected, hashEnabled, Optional.of(hashChannels.size())); } - @Test(dataProvider = "hashEnabled") + @Test(dataProvider = "hashEnabledValues") public void testHashAggregationWithGlobals(boolean hashEnabled) throws Exception { @@ -226,7 +203,7 @@ public void testHashAggregationWithGlobals(boolean hashEnabled) assertOperatorEqualsIgnoreOrder(operatorFactory, driverContext, input, expected, hashEnabled, Optional.of(groupByChannels.size())); } - @Test(dataProvider = "hashEnabled", expectedExceptions = ExceededMemoryLimitException.class, expectedExceptionsMessageRegExp = "Query exceeded local memory limit of 10B") + @Test(dataProvider = "hashEnabledValues", expectedExceptions = ExceededMemoryLimitException.class, expectedExceptionsMessageRegExp = "Query exceeded local memory limit of 10B") public void testMemoryLimit(boolean hashEnabled) { MetadataManager metadata = MetadataManager.createTestMetadataManager(); @@ -234,7 +211,7 @@ public void testMemoryLimit(boolean hashEnabled) new Signature("max", AGGREGATE, parseTypeSignature(StandardTypes.VARCHAR), parseTypeSignature(StandardTypes.VARCHAR))); List hashChannels = Ints.asList(1); - RowPagesBuilder rowPagesBuilder = rowPagesBuilder(hashEnabled, hashChannels, VARCHAR, BIGINT, VARCHAR, BIGINT); + RowPagesBuilder rowPagesBuilder = rowPagesBuilder(hashEnabled, hashChannels, VARCHAR, VARCHAR, VARCHAR, BIGINT); List input = rowPagesBuilder .addSequencePage(10, 100, 0, 100, 0) .addSequencePage(10, 100, 0, 200, 0) @@ -248,7 +225,7 @@ public void testMemoryLimit(boolean hashEnabled) HashAggregationOperatorFactory operatorFactory = new HashAggregationOperatorFactory( 0, new PlanNodeId("test"), - ImmutableList.of(BIGINT), + ImmutableList.of(VARCHAR), hashChannels, ImmutableList.of(), Step.SINGLE, @@ -264,8 +241,8 @@ public void testMemoryLimit(boolean hashEnabled) toPages(operatorFactory, driverContext, input); } - @Test(dataProvider = "hashEnabledAndMemoryLimitBeforeSpillValues") - public void testHashBuilderResize(boolean hashEnabled, long memoryLimitBeforeSpill, long memoryLimitForMergeWithMemory) + @Test(dataProvider = "hashEnabledValues") + public void testHashBuilderResize(boolean hashEnabled) { BlockBuilder builder = VARCHAR.createBlockBuilder(new BlockBuilderStatus(), 1, DEFAULT_MAX_BLOCK_SIZE_IN_BYTES); VARCHAR.writeSlice(builder, Slices.allocate(200_000)); // this must be larger than DEFAULT_MAX_BLOCK_SIZE, 64K @@ -294,16 +271,12 @@ public void testHashBuilderResize(boolean hashEnabled, long memoryLimitBeforeSpi rowPagesBuilder.getHashChannel(), Optional.empty(), 100_000, - new DataSize(16, MEGABYTE), - memoryLimitBeforeSpill > 0, - succinctBytes(memoryLimitBeforeSpill), - succinctBytes(memoryLimitForMergeWithMemory), - spillerFactory); + new DataSize(16, MEGABYTE)); toPages(operatorFactory, driverContext, input); } - @Test(dataProvider = "hashEnabled", expectedExceptions = ExceededMemoryLimitException.class, expectedExceptionsMessageRegExp = "Query exceeded local memory limit of 3MB") + @Test(dataProvider = "hashEnabledValues", expectedExceptions = ExceededMemoryLimitException.class, expectedExceptionsMessageRegExp = "Query exceeded local memory limit of 3MB") public void testHashBuilderResizeLimit(boolean hashEnabled) { BlockBuilder builder = VARCHAR.createBlockBuilder(new BlockBuilderStatus(), 1, DEFAULT_MAX_BLOCK_SIZE_IN_BYTES); @@ -338,7 +311,7 @@ public void testHashBuilderResizeLimit(boolean hashEnabled) toPages(operatorFactory, driverContext, input); } - @Test(dataProvider = "hashEnabled") + @Test(dataProvider = "hashEnabledValues") public void testMultiSliceAggregationOutput(boolean hashEnabled) { // estimate the number of entries required to create 1.5 pages of results @@ -369,8 +342,8 @@ public void testMultiSliceAggregationOutput(boolean hashEnabled) assertEquals(toPages(operatorFactory, driverContext, input).size(), 2); } - @Test(dataProvider = "hashEnabledAndMemoryLimitBeforeSpillValues") - public void testMultiplePartialFlushes(boolean hashEnabled, long memoryLimitBeforeSpill, long memoryLimitForMergeWithMemory) + @Test(dataProvider = "hashEnabledValues") + public void testMultiplePartialFlushes(boolean hashEnabled) throws Exception { List hashChannels = Ints.asList(0); @@ -393,11 +366,7 @@ public void testMultiplePartialFlushes(boolean hashEnabled, long memoryLimitBefo rowPagesBuilder.getHashChannel(), Optional.empty(), 100_000, - new DataSize(1, Unit.KILOBYTE), - memoryLimitBeforeSpill > 0, - succinctBytes(memoryLimitBeforeSpill), - succinctBytes(memoryLimitForMergeWithMemory), - spillerFactory); + new DataSize(1, Unit.KILOBYTE)); DriverContext driverContext = createTaskContext(executor, TEST_SESSION, new DataSize(4, Unit.KILOBYTE)) .addPipelineContext(true, true) @@ -451,148 +420,4 @@ public void testMultiplePartialFlushes(boolean hashEnabled, long memoryLimitBefo assertEqualsIgnoreOrder(actual.getMaterializedRows(), expected.getMaterializedRows()); } } - - @Test - public void testMergeWithMemorySpill() - { - List hashChannels = Ints.asList(0); - RowPagesBuilder rowPagesBuilder = rowPagesBuilder(BIGINT); - - int smallPagesSpillThresholdSize = 150000; - - List input = rowPagesBuilder - .addSequencePage(smallPagesSpillThresholdSize, 0) - .addSequencePage(10, smallPagesSpillThresholdSize) - .build(); - - HashAggregationOperatorFactory operatorFactory = new HashAggregationOperatorFactory( - 0, - new PlanNodeId("test"), - ImmutableList.of(BIGINT), - hashChannels, - ImmutableList.of(), - Step.SINGLE, - ImmutableList.of(LONG_SUM.bind(ImmutableList.of(0), Optional.empty())), - rowPagesBuilder.getHashChannel(), - Optional.empty(), - 1, - new DataSize(16, MEGABYTE), - true, - new DataSize(smallPagesSpillThresholdSize, Unit.BYTE), - succinctBytes(Integer.MAX_VALUE), - spillerFactory); - - DriverContext driverContext = createTaskContext(executor, TEST_SESSION, new DataSize(1, Unit.KILOBYTE)) - .addPipelineContext(true, true) - .addDriverContext(); - - MaterializedResult.Builder resultBuilder = resultBuilder(driverContext.getSession(), BIGINT); - for (int i = 0; i < smallPagesSpillThresholdSize + 10; ++i) { - resultBuilder.row((long) i, (long) i); - } - - assertOperatorEqualsIgnoreOrder(operatorFactory, driverContext, input, resultBuilder.build(), false, Optional.of(hashChannels.size())); - } - - @Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = ".* Failed to spill") - public void testSpillerFailure() - { - MetadataManager metadata = MetadataManager.createTestMetadataManager(); - InternalAggregationFunction maxVarcharColumn = metadata.getFunctionRegistry().getAggregateFunctionImplementation( - new Signature("max", AGGREGATE, parseTypeSignature(StandardTypes.VARCHAR), parseTypeSignature(StandardTypes.VARCHAR))); - - List hashChannels = Ints.asList(1); - RowPagesBuilder rowPagesBuilder = rowPagesBuilder(false, hashChannels, VARCHAR, BIGINT, VARCHAR, BIGINT); - List input = rowPagesBuilder - .addSequencePage(10, 100, 0, 100, 0) - .addSequencePage(10, 100, 0, 200, 0) - .addSequencePage(10, 100, 0, 300, 0) - .build(); - - DriverContext driverContext = createTaskContext(executor, TEST_SESSION, new DataSize(10, Unit.BYTE)) - .addPipelineContext(true, true) - .addDriverContext(); - - HashAggregationOperatorFactory operatorFactory = new HashAggregationOperatorFactory( - 0, - new PlanNodeId("test"), - ImmutableList.of(BIGINT), - hashChannels, - ImmutableList.of(), - Step.SINGLE, - ImmutableList.of(COUNT.bind(ImmutableList.of(0), Optional.empty()), - LONG_SUM.bind(ImmutableList.of(3), Optional.empty()), - LONG_AVERAGE.bind(ImmutableList.of(3), Optional.empty()), - maxVarcharColumn.bind(ImmutableList.of(2), Optional.empty())), - rowPagesBuilder.getHashChannel(), - Optional.empty(), - 100_000, - new DataSize(16, MEGABYTE), - true, - succinctBytes(8), - succinctBytes(Integer.MAX_VALUE), - new FailingSpillerFactory()); - - toPages(operatorFactory, driverContext, input); - } - - private static class DummySpillerFactory - extends SpillerFactoryWithStats - { - @Override - public Spiller create(List types) - { - return new Spiller() - { - private final List> spills = new ArrayList<>(); - - @Override - public CompletableFuture spill(Iterator pageIterator) - { - spills.add(pageIterator); - return CompletableFuture.completedFuture(null); - } - - @Override - public List> getSpills() - { - return spills; - } - - @Override - public void close() - { - } - }; - } - } - - private static class FailingSpillerFactory - extends SpillerFactoryWithStats - { - @Override - public Spiller create(List types) - { - return new Spiller() { - @Override - public CompletableFuture spill(Iterator pageIterator) - { - SettableFuture future = SettableFuture.create(); - future.setException(new IOException("Failed to spill")); - return toCompletableFuture(future); - } - - @Override - public List> getSpills() - { - return ImmutableList.of(); - } - - @Override - public void close() - { - } - }; - } - } } diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestHashJoinOperator.java b/presto-main/src/test/java/com/facebook/presto/operator/TestHashJoinOperator.java index 48aec2ae333b..62a5798484f3 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestHashJoinOperator.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestHashJoinOperator.java @@ -99,7 +99,7 @@ public void testInnerJoin(boolean parallelBuild, boolean probeHashEnabled, boole LookupSourceFactory lookupSourceFactory = buildHash(parallelBuild, taskContext, Ints.asList(0), buildPages, Optional.empty()); // probe - RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)); + RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)); List probeInput = probePages .addSequencePage(1000, 0, 1000, 2000) .build(); @@ -137,7 +137,7 @@ public void testInnerJoinWithNullProbe(boolean parallelBuild, boolean probeHashE TaskContext taskContext = createTaskContext(); // build - List buildTypes = ImmutableList.of(VARCHAR); + List buildTypes = ImmutableList.of(VARCHAR); RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes) .row("a") .row("b") @@ -145,7 +145,7 @@ public void testInnerJoinWithNullProbe(boolean parallelBuild, boolean probeHashE LookupSourceFactory lookupSourceFactory = buildHash(parallelBuild, taskContext, Ints.asList(0), buildPages, Optional.empty()); // probe - List probeTypes = ImmutableList.of(VARCHAR); + List probeTypes = ImmutableList.of(VARCHAR); RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); List probeInput = probePages .row("a") @@ -181,7 +181,7 @@ public void testInnerJoinWithNullBuild(boolean parallelBuild, boolean probeHashE TaskContext taskContext = createTaskContext(); // build - List buildTypes = ImmutableList.of(VARCHAR); + List buildTypes = ImmutableList.of(VARCHAR); RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes) .row("a") .row((String) null) @@ -191,7 +191,7 @@ public void testInnerJoinWithNullBuild(boolean parallelBuild, boolean probeHashE LookupSourceFactory lookupSourceFactory = buildHash(parallelBuild, taskContext, Ints.asList(0), buildPages, Optional.empty()); // probe - List probeTypes = ImmutableList.of(VARCHAR); + List probeTypes = ImmutableList.of(VARCHAR); RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); List probeInput = probePages .row("a") @@ -225,7 +225,7 @@ public void testInnerJoinWithNullOnBothSides(boolean parallelBuild, boolean prob TaskContext taskContext = createTaskContext(); // build - List buildTypes = ImmutableList.of(VARCHAR); + List buildTypes = ImmutableList.of(VARCHAR); RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes) .row("a") .row((String) null) @@ -235,7 +235,7 @@ public void testInnerJoinWithNullOnBothSides(boolean parallelBuild, boolean prob LookupSourceFactory lookupSourceFactory = buildHash(parallelBuild, taskContext, Ints.asList(0), buildPages, Optional.empty()); // probe - List probeTypes = ImmutableList.of(VARCHAR); + List probeTypes = ImmutableList.of(VARCHAR); RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); List probeInput = probePages .row("a") @@ -270,13 +270,13 @@ public void testProbeOuterJoin(boolean parallelBuild, boolean probeHashEnabled, TaskContext taskContext = createTaskContext(); // build - List buildTypes = ImmutableList.of(VARCHAR, BIGINT, BIGINT); + List buildTypes = ImmutableList.of(VARCHAR, BIGINT, BIGINT); RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)) .addSequencePage(10, 20, 30, 40); LookupSourceFactory lookupSourceFactory = buildHash(parallelBuild, taskContext, Ints.asList(0), buildPages, Optional.empty()); // probe - List probeTypes = ImmutableList.of(VARCHAR, BIGINT, BIGINT); + List probeTypes = ImmutableList.of(VARCHAR, BIGINT, BIGINT); RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); List probeInput = probePages .addSequencePage(15, 20, 1020, 2020) @@ -323,13 +323,13 @@ public void testProbeOuterJoinWithFilterFunction(boolean parallelBuild, boolean (leftPosition, leftBlocks, rightPosition, rightBlocks) -> BIGINT.getLong(rightBlocks[1], rightPosition) >= 1025)); // build - List buildTypes = ImmutableList.of(VARCHAR, BIGINT, BIGINT); + List buildTypes = ImmutableList.of(VARCHAR, BIGINT, BIGINT); RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR, BIGINT, BIGINT)) .addSequencePage(10, 20, 30, 40); LookupSourceFactory lookupSourceFactory = buildHash(parallelBuild, taskContext, Ints.asList(0), buildPages, Optional.of(filterFunction)); // probe - List probeTypes = ImmutableList.of(VARCHAR, BIGINT, BIGINT); + List probeTypes = ImmutableList.of(VARCHAR, BIGINT, BIGINT); RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); List probeInput = probePages .addSequencePage(15, 20, 1020, 2020) @@ -372,7 +372,7 @@ public void testOuterJoinWithNullProbe(boolean parallelBuild, boolean probeHashE TaskContext taskContext = createTaskContext(); // build - List buildTypes = ImmutableList.of(VARCHAR); + List buildTypes = ImmutableList.of(VARCHAR); RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes) .row("a") .row("b") @@ -380,7 +380,7 @@ public void testOuterJoinWithNullProbe(boolean parallelBuild, boolean probeHashE LookupSourceFactory lookupSourceFactory = buildHash(parallelBuild, taskContext, Ints.asList(0), buildPages, Optional.empty()); // probe - List probeTypes = ImmutableList.of(VARCHAR); + List probeTypes = ImmutableList.of(VARCHAR); RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); List probeInput = probePages .row("a") @@ -420,7 +420,7 @@ public void testOuterJoinWithNullProbeAndFilterFunction(boolean parallelBuild, b (leftPosition, leftBlocks, rightPosition, rightBlocks) -> VARCHAR.getSlice(rightBlocks[0], rightPosition).toStringAscii().equals("a"))); // build - List buildTypes = ImmutableList.of(VARCHAR); + List buildTypes = ImmutableList.of(VARCHAR); RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), buildTypes) .row("a") .row("b") @@ -428,7 +428,7 @@ public void testOuterJoinWithNullProbeAndFilterFunction(boolean parallelBuild, b LookupSourceFactory lookupSourceFactory = buildHash(parallelBuild, taskContext, Ints.asList(0), buildPages, Optional.of(filterFunction)); // probe - List probeTypes = ImmutableList.of(VARCHAR); + List probeTypes = ImmutableList.of(VARCHAR); RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); List probeInput = probePages .row("a") @@ -465,7 +465,7 @@ public void testOuterJoinWithNullBuild(boolean parallelBuild, boolean probeHashE TaskContext taskContext = createTaskContext(); // build - List buildTypes = ImmutableList.of(VARCHAR); + List buildTypes = ImmutableList.of(VARCHAR); RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR)) .row("a") .row((String) null) @@ -475,7 +475,7 @@ public void testOuterJoinWithNullBuild(boolean parallelBuild, boolean probeHashE LookupSourceFactory lookupSourceFactory = buildHash(parallelBuild, taskContext, Ints.asList(0), buildPages, Optional.empty()); // probe - List probeTypes = ImmutableList.of(VARCHAR); + List probeTypes = ImmutableList.of(VARCHAR); RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); List probeInput = probePages .row("a") @@ -513,7 +513,7 @@ public void testOuterJoinWithNullBuildAndFilterFunction(boolean parallelBuild, b ImmutableSet.of("a", "c").contains(VARCHAR.getSlice(rightBlocks[0], rightPosition).toStringAscii()))); // build - List buildTypes = ImmutableList.of(VARCHAR); + List buildTypes = ImmutableList.of(VARCHAR); RowPagesBuilder buildPages = rowPagesBuilder(buildHashEnabled, Ints.asList(0), ImmutableList.of(VARCHAR)) .row("a") .row((String) null) @@ -523,7 +523,7 @@ public void testOuterJoinWithNullBuildAndFilterFunction(boolean parallelBuild, b LookupSourceFactory lookupSourceFactory = buildHash(parallelBuild, taskContext, Ints.asList(0), buildPages, Optional.of(filterFunction)); // probe - List probeTypes = ImmutableList.of(VARCHAR); + List probeTypes = ImmutableList.of(VARCHAR); RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); List probeInput = probePages .row("a") @@ -566,7 +566,7 @@ public void testOuterJoinWithNullOnBothSides(boolean parallelBuild, boolean prob LookupSourceFactory lookupSourceFactory = buildHash(parallelBuild, taskContext, Ints.asList(0), buildPages, Optional.empty()); // probe - List probeTypes = ImmutableList.of(VARCHAR); + List probeTypes = ImmutableList.of(VARCHAR); RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); List probeInput = probePages .row("a") @@ -615,7 +615,7 @@ public void testOuterJoinWithNullOnBothSidesAndFilterFunction(boolean parallelBu LookupSourceFactory lookupSourceFactory = buildHash(parallelBuild, taskContext, Ints.asList(0), buildPages, Optional.of(filterFunction)); // probe - List probeTypes = ImmutableList.of(VARCHAR); + List probeTypes = ImmutableList.of(VARCHAR); RowPagesBuilder probePages = rowPagesBuilder(probeHashEnabled, Ints.asList(0), probeTypes); List probeInput = probePages .row("a") diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestHashSemiJoinOperator.java b/presto-main/src/test/java/com/facebook/presto/operator/TestHashSemiJoinOperator.java index 97f678b28dad..30bd55ef9b62 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestHashSemiJoinOperator.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestHashSemiJoinOperator.java @@ -95,7 +95,7 @@ public void testSemiJoin(boolean hashEnabled) } // probe - List probeTypes = ImmutableList.of(BIGINT, BIGINT); + List probeTypes = ImmutableList.of(BIGINT, BIGINT); RowPagesBuilder rowPagesBuilderProbe = rowPagesBuilder(hashEnabled, Ints.asList(0), BIGINT, BIGINT); List probeInput = rowPagesBuilderProbe .addSequencePage(10, 30, 0) @@ -132,7 +132,7 @@ public void testBuildSideNulls(boolean hashEnabled) // build OperatorContext operatorContext = driverContext.addOperatorContext(0, new PlanNodeId("test"), ValuesOperator.class.getSimpleName()); - List buildTypes = ImmutableList.of(BIGINT); + List buildTypes = ImmutableList.of(BIGINT); RowPagesBuilder rowPagesBuilder = rowPagesBuilder(hashEnabled, Ints.asList(0), buildTypes); Operator buildOperator = new ValuesOperator(operatorContext, buildTypes, rowPagesBuilder .row(0L) @@ -151,7 +151,7 @@ public void testBuildSideNulls(boolean hashEnabled) } // probe - List probeTypes = ImmutableList.of(BIGINT); + List probeTypes = ImmutableList.of(BIGINT); RowPagesBuilder rowPagesBuilderProbe = rowPagesBuilder(hashEnabled, Ints.asList(0), probeTypes); List probeInput = rowPagesBuilderProbe .addSequencePage(4, 1) @@ -182,7 +182,7 @@ public void testProbeSideNulls(boolean hashEnabled) // build OperatorContext operatorContext = driverContext.addOperatorContext(0, new PlanNodeId("test"), ValuesOperator.class.getSimpleName()); - List buildTypes = ImmutableList.of(BIGINT); + List buildTypes = ImmutableList.of(BIGINT); RowPagesBuilder rowPagesBuilder = rowPagesBuilder(hashEnabled, Ints.asList(0), buildTypes); Operator buildOperator = new ValuesOperator(operatorContext, buildTypes, rowPagesBuilder .row(0L) @@ -198,7 +198,7 @@ public void testProbeSideNulls(boolean hashEnabled) } // probe - List probeTypes = ImmutableList.of(BIGINT); + List probeTypes = ImmutableList.of(BIGINT); RowPagesBuilder rowPagesBuilderProbe = rowPagesBuilder(hashEnabled, Ints.asList(0), probeTypes); List probeInput = rowPagesBuilderProbe .row(0L) @@ -232,7 +232,7 @@ public void testProbeAndBuildNulls(boolean hashEnabled) // build OperatorContext operatorContext = driverContext.addOperatorContext(0, new PlanNodeId("test"), ValuesOperator.class.getSimpleName()); - List buildTypes = ImmutableList.of(BIGINT); + List buildTypes = ImmutableList.of(BIGINT); RowPagesBuilder rowPagesBuilder = rowPagesBuilder(hashEnabled, Ints.asList(0), buildTypes); Operator buildOperator = new ValuesOperator(operatorContext, buildTypes, rowPagesBuilder .row(0L) @@ -249,7 +249,7 @@ public void testProbeAndBuildNulls(boolean hashEnabled) } // probe - List probeTypes = ImmutableList.of(BIGINT); + List probeTypes = ImmutableList.of(BIGINT); RowPagesBuilder rowPagesBuilderProbe = rowPagesBuilder(hashEnabled, Ints.asList(0), probeTypes); List probeInput = rowPagesBuilderProbe .row(0L) @@ -284,7 +284,7 @@ public void testMemoryLimit(boolean hashEnabled) .addDriverContext(); OperatorContext operatorContext = driverContext.addOperatorContext(0, new PlanNodeId("test"), ValuesOperator.class.getSimpleName()); - List buildTypes = ImmutableList.of(BIGINT); + List buildTypes = ImmutableList.of(BIGINT); RowPagesBuilder rowPagesBuilder = rowPagesBuilder(hashEnabled, Ints.asList(0), buildTypes); Operator buildOperator = new ValuesOperator(operatorContext, buildTypes, rowPagesBuilder .addSequencePage(10000, 20) diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestHttpPageBufferClient.java b/presto-main/src/test/java/com/facebook/presto/operator/TestHttpPageBufferClient.java index 1a1d6b931208..9e6d9164236b 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestHttpPageBufferClient.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestHttpPageBufferClient.java @@ -171,7 +171,7 @@ public void testLifecycle() CyclicBarrier beforeRequest = new CyclicBarrier(2); CyclicBarrier afterRequest = new CyclicBarrier(2); StaticRequestProcessor processor = new StaticRequestProcessor(beforeRequest, afterRequest); - processor.setResponse(new TestingResponse(HttpStatus.NO_CONTENT, ImmutableListMultimap.of(), new byte[0])); + processor.setResponse(new TestingResponse(HttpStatus.NO_CONTENT, ImmutableListMultimap.of(), new byte[0])); CyclicBarrier requestComplete = new CyclicBarrier(2); TestingClientCallback callback = new TestingClientCallback(requestComplete); @@ -279,7 +279,7 @@ public void testCloseDuringPendingRequest() CyclicBarrier beforeRequest = new CyclicBarrier(2); CyclicBarrier afterRequest = new CyclicBarrier(2); StaticRequestProcessor processor = new StaticRequestProcessor(beforeRequest, afterRequest); - processor.setResponse(new TestingResponse(HttpStatus.NO_CONTENT, ImmutableListMultimap.of(), new byte[0])); + processor.setResponse(new TestingResponse(HttpStatus.NO_CONTENT, ImmutableListMultimap.of(), new byte[0])); CyclicBarrier requestComplete = new CyclicBarrier(2); TestingClientCallback callback = new TestingClientCallback(requestComplete); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestMergeHashSort.java b/presto-main/src/test/java/com/facebook/presto/operator/TestMergeHashSort.java index dcb0a24e6b38..c54a6f532573 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestMergeHashSort.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestMergeHashSort.java @@ -127,7 +127,7 @@ public void testPageRewriteIterator() assertPageEquals(types, pages.get(0), expectedPages.get(0)); } - private static long readBigint(MergeHashSort.PagePosition pagePosition) + private long readBigint(MergeHashSort.PagePosition pagePosition) { return BIGINT.getLong(pagePosition.getPage().getBlock(0), pagePosition.getPosition()); } diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestNestedLoopJoinOperator.java b/presto-main/src/test/java/com/facebook/presto/operator/TestNestedLoopJoinOperator.java index db41e12119d1..9ff3dcfd6683 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestNestedLoopJoinOperator.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestNestedLoopJoinOperator.java @@ -71,11 +71,11 @@ public void testNestedLoopJoin() NestedLoopJoinPagesSupplier nestedLoopJoinPagesSupplier = buildPageSource(taskContext, buildPages); // probe - RowPagesBuilder probePages = rowPagesBuilder(ImmutableList.of(VARCHAR, BIGINT, BIGINT)); + RowPagesBuilder probePages = rowPagesBuilder(ImmutableList.of(VARCHAR, BIGINT, BIGINT)); List probeInput = probePages .addSequencePage(2, 0, 1000, 2000) .build(); - NestedLoopJoinOperatorFactory joinOperatorFactory = new NestedLoopJoinOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinPagesSupplier, ImmutableList.of(VARCHAR, BIGINT, BIGINT)); + NestedLoopJoinOperatorFactory joinOperatorFactory = new NestedLoopJoinOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinPagesSupplier, ImmutableList.of(VARCHAR, BIGINT, BIGINT)); // expected MaterializedResult expected = resultBuilder(taskContext.getSession(), concat(probePages.getTypes(), buildPages.getTypes())) @@ -93,10 +93,10 @@ public void testNestedLoopJoin() buildPages = rowPagesBuilder(ImmutableList.of(VARCHAR, BIGINT, BIGINT)) .addSequencePage(2, 20, 30, 40); nestedLoopJoinPagesSupplier = buildPageSource(taskContext, buildPages); - joinOperatorFactory = new NestedLoopJoinOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinPagesSupplier, ImmutableList.of(VARCHAR, BIGINT, BIGINT)); + joinOperatorFactory = new NestedLoopJoinOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinPagesSupplier, ImmutableList.of(VARCHAR, BIGINT, BIGINT)); // probe - probePages = rowPagesBuilder(ImmutableList.of(VARCHAR, BIGINT, BIGINT)); + probePages = rowPagesBuilder(ImmutableList.of(VARCHAR, BIGINT, BIGINT)); probeInput = probePages .addSequencePage(3, 0, 1000, 2000) .build(); @@ -121,14 +121,14 @@ public void testCrossJoinWithNullProbe() TaskContext taskContext = createTaskContext(); // build - List buildTypes = ImmutableList.of(VARCHAR); + List buildTypes = ImmutableList.of(VARCHAR); RowPagesBuilder buildPages = rowPagesBuilder(buildTypes) .row("a") .row("b"); NestedLoopJoinPagesSupplier nestedLoopJoinPagesSupplier = buildPageSource(taskContext, buildPages); // probe - List probeTypes = ImmutableList.of(VARCHAR); + List probeTypes = ImmutableList.of(VARCHAR); RowPagesBuilder probePages = rowPagesBuilder(probeTypes); List probeInput = probePages .row("A") @@ -138,7 +138,7 @@ public void testCrossJoinWithNullProbe() .row("B") .build(); - NestedLoopJoinOperatorFactory joinOperatorFactory = new NestedLoopJoinOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinPagesSupplier, ImmutableList.of(VARCHAR)); + NestedLoopJoinOperatorFactory joinOperatorFactory = new NestedLoopJoinOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinPagesSupplier, ImmutableList.of(VARCHAR)); // expected MaterializedResult expected = resultBuilder(taskContext.getSession(), concat(probeTypes, buildPages.getTypes())) @@ -164,7 +164,7 @@ public void testCrossJoinWithNullBuild() TaskContext taskContext = createTaskContext(); // build - List buildTypes = ImmutableList.of(VARCHAR); + List buildTypes = ImmutableList.of(VARCHAR); RowPagesBuilder buildPages = rowPagesBuilder(buildTypes) .row("a") .row((String) null) @@ -174,13 +174,13 @@ public void testCrossJoinWithNullBuild() NestedLoopJoinPagesSupplier nestedLoopJoinPagesSupplier = buildPageSource(taskContext, buildPages); // probe - List probeTypes = ImmutableList.of(VARCHAR); + List probeTypes = ImmutableList.of(VARCHAR); RowPagesBuilder probePages = rowPagesBuilder(probeTypes); List probeInput = probePages .row("A") .row("B") .build(); - NestedLoopJoinOperatorFactory joinOperatorFactory = new NestedLoopJoinOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinPagesSupplier, ImmutableList.of(VARCHAR)); + NestedLoopJoinOperatorFactory joinOperatorFactory = new NestedLoopJoinOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinPagesSupplier, ImmutableList.of(VARCHAR)); // expected MaterializedResult expected = resultBuilder(taskContext.getSession(), concat(probeTypes, buildPages.getTypes())) @@ -206,7 +206,7 @@ public void testCrossJoinWithNullOnBothSides() TaskContext taskContext = createTaskContext(); // build - List buildTypes = ImmutableList.of(VARCHAR); + List buildTypes = ImmutableList.of(VARCHAR); RowPagesBuilder buildPages = rowPagesBuilder(buildTypes) .row("a") .row((String) null) @@ -216,7 +216,7 @@ public void testCrossJoinWithNullOnBothSides() NestedLoopJoinPagesSupplier nestedLoopJoinPagesSupplier = buildPageSource(taskContext, buildPages); // probe - List probeTypes = ImmutableList.of(VARCHAR); + List probeTypes = ImmutableList.of(VARCHAR); RowPagesBuilder probePages = rowPagesBuilder(probeTypes); List probeInput = probePages .row("A") @@ -224,7 +224,7 @@ public void testCrossJoinWithNullOnBothSides() .row((String) null) .row("C") .build(); - NestedLoopJoinOperatorFactory joinOperatorFactory = new NestedLoopJoinOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinPagesSupplier, ImmutableList.of(VARCHAR)); + NestedLoopJoinOperatorFactory joinOperatorFactory = new NestedLoopJoinOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinPagesSupplier, ImmutableList.of(VARCHAR)); // expected MaterializedResult expected = resultBuilder(taskContext.getSession(), concat(probeTypes, buildPages.getTypes())) @@ -260,7 +260,7 @@ public void testBuildMultiplePages() TaskContext taskContext = createTaskContext(); // build - List buildTypes = ImmutableList.of(VARCHAR); + List buildTypes = ImmutableList.of(VARCHAR); RowPagesBuilder buildPages = rowPagesBuilder(buildTypes) .row("a") .pageBreak() @@ -272,13 +272,13 @@ public void testBuildMultiplePages() NestedLoopJoinPagesSupplier nestedLoopJoinPagesSupplier = buildPageSource(taskContext, buildPages); // probe - List probeTypes = ImmutableList.of(VARCHAR); + List probeTypes = ImmutableList.of(VARCHAR); RowPagesBuilder probePages = rowPagesBuilder(probeTypes); List probeInput = probePages .row("A") .row("B") .build(); - NestedLoopJoinOperatorFactory joinOperatorFactory = new NestedLoopJoinOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinPagesSupplier, ImmutableList.of(VARCHAR)); + NestedLoopJoinOperatorFactory joinOperatorFactory = new NestedLoopJoinOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinPagesSupplier, ImmutableList.of(VARCHAR)); // expected MaterializedResult expected = resultBuilder(taskContext.getSession(), concat(probeTypes, buildPages.getTypes())) @@ -304,14 +304,14 @@ public void testProbeMultiplePages() TaskContext taskContext = createTaskContext(); // build - List buildTypes = ImmutableList.of(VARCHAR); + List buildTypes = ImmutableList.of(VARCHAR); RowPagesBuilder buildPages = rowPagesBuilder(buildTypes) .row("A") .row("B"); NestedLoopJoinPagesSupplier nestedLoopJoinPagesSupplier = buildPageSource(taskContext, buildPages); // probe - List probeTypes = ImmutableList.of(VARCHAR); + List probeTypes = ImmutableList.of(VARCHAR); RowPagesBuilder probePages = rowPagesBuilder(probeTypes); List probeInput = probePages .row("a") @@ -322,7 +322,7 @@ public void testProbeMultiplePages() .pageBreak() .row("d") .build(); - NestedLoopJoinOperatorFactory joinOperatorFactory = new NestedLoopJoinOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinPagesSupplier, ImmutableList.of(VARCHAR)); + NestedLoopJoinOperatorFactory joinOperatorFactory = new NestedLoopJoinOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinPagesSupplier, ImmutableList.of(VARCHAR)); // expected MaterializedResult expected = resultBuilder(taskContext.getSession(), concat(probeTypes, buildPages.getTypes())) @@ -348,7 +348,7 @@ public void testProbeAndBuildMultiplePages() TaskContext taskContext = createTaskContext(); // build - List buildTypes = ImmutableList.of(VARCHAR); + List buildTypes = ImmutableList.of(VARCHAR); RowPagesBuilder buildPages = rowPagesBuilder(buildTypes) .row("A") .row("B") @@ -357,7 +357,7 @@ public void testProbeAndBuildMultiplePages() NestedLoopJoinPagesSupplier nestedLoopJoinPagesSupplier = buildPageSource(taskContext, buildPages); // probe - List probeTypes = ImmutableList.of(VARCHAR); + List probeTypes = ImmutableList.of(VARCHAR); RowPagesBuilder probePages = rowPagesBuilder(probeTypes); List probeInput = probePages .row("a") @@ -368,7 +368,7 @@ public void testProbeAndBuildMultiplePages() .pageBreak() .row("d") .build(); - NestedLoopJoinOperatorFactory joinOperatorFactory = new NestedLoopJoinOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinPagesSupplier, ImmutableList.of(VARCHAR)); + NestedLoopJoinOperatorFactory joinOperatorFactory = new NestedLoopJoinOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinPagesSupplier, ImmutableList.of(VARCHAR)); // expected MaterializedResult expected = resultBuilder(taskContext.getSession(), concat(probeTypes, buildPages.getTypes())) @@ -399,7 +399,7 @@ public void testEmptyProbePage() TaskContext taskContext = createTaskContext(); // build - List buildTypes = ImmutableList.of(VARCHAR); + List buildTypes = ImmutableList.of(VARCHAR); RowPagesBuilder buildPages = rowPagesBuilder(buildTypes) .row("A") .row("B") @@ -408,12 +408,12 @@ public void testEmptyProbePage() NestedLoopJoinPagesSupplier nestedLoopJoinPagesSupplier = buildPageSource(taskContext, buildPages); // probe - List probeTypes = ImmutableList.of(VARCHAR); + List probeTypes = ImmutableList.of(VARCHAR); RowPagesBuilder probePages = rowPagesBuilder(probeTypes); List probeInput = probePages .pageBreak() .build(); - NestedLoopJoinOperatorFactory joinOperatorFactory = new NestedLoopJoinOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinPagesSupplier, ImmutableList.of(VARCHAR)); + NestedLoopJoinOperatorFactory joinOperatorFactory = new NestedLoopJoinOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinPagesSupplier, ImmutableList.of(VARCHAR)); // expected MaterializedResult expected = resultBuilder(taskContext.getSession(), concat(probeTypes, buildPages.getTypes())) @@ -429,20 +429,20 @@ public void testEmptyBuildPage() TaskContext taskContext = createTaskContext(); // build - List buildTypes = ImmutableList.of(VARCHAR); + List buildTypes = ImmutableList.of(VARCHAR); RowPagesBuilder buildPages = rowPagesBuilder(buildTypes) .pageBreak(); NestedLoopJoinPagesSupplier nestedLoopJoinPagesSupplier = buildPageSource(taskContext, buildPages); // probe - List probeTypes = ImmutableList.of(VARCHAR); + List probeTypes = ImmutableList.of(VARCHAR); RowPagesBuilder probePages = rowPagesBuilder(probeTypes); List probeInput = probePages .row("A") .row("B") .pageBreak() .build(); - NestedLoopJoinOperatorFactory joinOperatorFactory = new NestedLoopJoinOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinPagesSupplier, ImmutableList.of(VARCHAR)); + NestedLoopJoinOperatorFactory joinOperatorFactory = new NestedLoopJoinOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinPagesSupplier, ImmutableList.of(VARCHAR)); // expected MaterializedResult expected = resultBuilder(taskContext.getSession(), concat(probeTypes, buildPages.getTypes())) diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestRowNumberOperator.java b/presto-main/src/test/java/com/facebook/presto/operator/TestRowNumberOperator.java index 8c1baacc3b45..a0608f63973b 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestRowNumberOperator.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestRowNumberOperator.java @@ -18,6 +18,7 @@ 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.Type; import com.facebook.presto.sql.planner.plan.PlanNodeId; import com.facebook.presto.testing.MaterializedResult; import com.google.common.collect.ImmutableList; @@ -105,7 +106,7 @@ public void testRowNumberUnpartitioned() ImmutableList.of(BIGINT, DOUBLE), Ints.asList(1, 0), Ints.asList(), - ImmutableList.of(), + ImmutableList.of(), Optional.empty(), Optional.empty(), 10); @@ -294,7 +295,7 @@ public void testRowNumberUnpartitionedLimit() ImmutableList.of(BIGINT, DOUBLE), Ints.asList(1, 0), Ints.asList(), - ImmutableList.of(), + ImmutableList.of(), Optional.of(3), Optional.empty(), 10); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestScanFilterAndProjectOperator.java b/presto-main/src/test/java/com/facebook/presto/operator/TestScanFilterAndProjectOperator.java index f7724d5dbaef..24fcfedb5d96 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestScanFilterAndProjectOperator.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestScanFilterAndProjectOperator.java @@ -23,6 +23,7 @@ import com.facebook.presto.spi.FixedPageSource; import com.facebook.presto.spi.Page; import com.facebook.presto.spi.RecordPageSource; +import com.facebook.presto.spi.type.Type; import com.facebook.presto.split.PageSourceProvider; import com.facebook.presto.sql.planner.plan.PlanNodeId; import com.facebook.presto.testing.MaterializedResult; @@ -73,15 +74,15 @@ public ConnectorPageSource createPageSource(Session session, Split split, List new GenericCursorProcessor(FilterFunctions.TRUE_FUNCTION, ImmutableList.of(singleColumn(VARCHAR, 0))), () -> new GenericPageProcessor(FilterFunctions.TRUE_FUNCTION, ImmutableList.of(singleColumn(VARCHAR, 0))), - ImmutableList.of(), - ImmutableList.of(VARCHAR)); + ImmutableList.of(), + ImmutableList.of(VARCHAR)); SourceOperator operator = factory.createOperator(driverContext); operator.addSplit(new Split(new ConnectorId("test"), TestingTransactionHandle.create(), TestingSplit.createLocalSplit())); operator.noMoreSplits(); - MaterializedResult expected = toMaterializedResult(driverContext.getSession(), ImmutableList.of(VARCHAR), ImmutableList.of(input)); - MaterializedResult actual = toMaterializedResult(driverContext.getSession(), ImmutableList.of(VARCHAR), toPages(operator)); + MaterializedResult expected = toMaterializedResult(driverContext.getSession(), ImmutableList.of(VARCHAR), ImmutableList.of(input)); + MaterializedResult actual = toMaterializedResult(driverContext.getSession(), ImmutableList.of(VARCHAR), toPages(operator)); assertEquals(actual.getRowCount(), expected.getRowCount()); assertEquals(actual, expected); @@ -102,20 +103,20 @@ public void testRecordCursorSource() @Override public ConnectorPageSource createPageSource(Session session, Split split, List columns) { - return new RecordPageSource(new PageRecordSet(ImmutableList.of(VARCHAR), input)); + return new RecordPageSource(new PageRecordSet(ImmutableList.of(VARCHAR), input)); } }, () -> new GenericCursorProcessor(FilterFunctions.TRUE_FUNCTION, ImmutableList.of(singleColumn(VARCHAR, 0))), () -> new GenericPageProcessor(FilterFunctions.TRUE_FUNCTION, ImmutableList.of(singleColumn(VARCHAR, 0))), - ImmutableList.of(), - ImmutableList.of(VARCHAR)); + ImmutableList.of(), + ImmutableList.of(VARCHAR)); SourceOperator operator = factory.createOperator(driverContext); operator.addSplit(new Split(new ConnectorId("test"), TestingTransactionHandle.create(), TestingSplit.createLocalSplit())); operator.noMoreSplits(); - MaterializedResult expected = toMaterializedResult(driverContext.getSession(), ImmutableList.of(VARCHAR), ImmutableList.of(input)); - MaterializedResult actual = toMaterializedResult(driverContext.getSession(), ImmutableList.of(VARCHAR), toPages(operator)); + MaterializedResult expected = toMaterializedResult(driverContext.getSession(), ImmutableList.of(VARCHAR), ImmutableList.of(input)); + MaterializedResult actual = toMaterializedResult(driverContext.getSession(), ImmutableList.of(VARCHAR), toPages(operator)); assertEquals(actual.getRowCount(), expected.getRowCount()); assertEquals(actual, expected); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestTopNRowNumberOperator.java b/presto-main/src/test/java/com/facebook/presto/operator/TestTopNRowNumberOperator.java index bbec9ee3786c..3b4aa514268f 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestTopNRowNumberOperator.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestTopNRowNumberOperator.java @@ -16,6 +16,7 @@ import com.facebook.presto.RowPagesBuilder; import com.facebook.presto.spi.Page; import com.facebook.presto.spi.block.SortOrder; +import com.facebook.presto.spi.type.Type; import com.facebook.presto.sql.planner.plan.PlanNodeId; import com.facebook.presto.testing.MaterializedResult; import com.google.common.collect.ImmutableList; @@ -142,7 +143,7 @@ public void testTopNRowNumberUnPartitioned() ImmutableList.of(BIGINT, DOUBLE), Ints.asList(1, 0), Ints.asList(), - ImmutableList.of(), + ImmutableList.of(), Ints.asList(1), ImmutableList.of(SortOrder.ASC_NULLS_LAST), 3, diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestUnnestOperator.java b/presto-main/src/test/java/com/facebook/presto/operator/TestUnnestOperator.java index 1890e15c7095..59f65aed68fe 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestUnnestOperator.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestUnnestOperator.java @@ -84,7 +84,7 @@ public void testUnnest() .build(); OperatorFactory operatorFactory = new UnnestOperator.UnnestOperatorFactory( - 0, new PlanNodeId("test"), ImmutableList.of(0), ImmutableList.of(BIGINT), ImmutableList.of(1, 2), ImmutableList.of(arrayType, mapType), false); + 0, new PlanNodeId("test"), ImmutableList.of(0), ImmutableList.of(BIGINT), ImmutableList.of(1, 2), ImmutableList.of(arrayType, mapType), false); MaterializedResult expected = resultBuilder(driverContext.getSession(), BIGINT, BIGINT, BIGINT, BIGINT) .row(1L, 2L, 4L, 5L) @@ -120,7 +120,7 @@ public void testUnnestWithArray() .build(); OperatorFactory operatorFactory = new UnnestOperator.UnnestOperatorFactory( - 0, new PlanNodeId("test"), ImmutableList.of(0), ImmutableList.of(BIGINT), ImmutableList.of(1, 2), ImmutableList.of(arrayType, mapType), false); + 0, new PlanNodeId("test"), ImmutableList.of(0), ImmutableList.of(BIGINT), ImmutableList.of(1, 2), ImmutableList.of(arrayType, mapType), false); MaterializedResult expected = resultBuilder(driverContext.getSession(), BIGINT, new ArrayType(BIGINT), new ArrayType(BIGINT), new ArrayType(BIGINT)) .row(1L, ImmutableList.of(2L, 4L), ImmutableList.of(4L, 8L), ImmutableList.of(5L, 10L)) @@ -150,7 +150,7 @@ public void testUnnestWithOrdinality() .build(); OperatorFactory operatorFactory = new UnnestOperator.UnnestOperatorFactory( - 0, new PlanNodeId("test"), ImmutableList.of(0), ImmutableList.of(BIGINT), ImmutableList.of(1, 2), ImmutableList.of(arrayType, mapType), true); + 0, new PlanNodeId("test"), ImmutableList.of(0), ImmutableList.of(BIGINT), ImmutableList.of(1, 2), ImmutableList.of(arrayType, mapType), true); MaterializedResult expected = resultBuilder(driverContext.getSession(), BIGINT, BIGINT, BIGINT, BIGINT, BIGINT) .row(1L, 2L, 4L, 5L, 1L) @@ -177,7 +177,7 @@ public void testUnnestNonNumericDoubles() .build(); OperatorFactory operatorFactory = new UnnestOperator.UnnestOperatorFactory( - 0, new PlanNodeId("test"), ImmutableList.of(0), ImmutableList.of(BIGINT), ImmutableList.of(1, 2), ImmutableList.of(arrayType, mapType), false); + 0, new PlanNodeId("test"), ImmutableList.of(0), ImmutableList.of(BIGINT), ImmutableList.of(1, 2), ImmutableList.of(arrayType, mapType), false); MaterializedResult expected = resultBuilder(driverContext.getSession(), BIGINT, DOUBLE, BIGINT, DOUBLE) .row(1L, NEGATIVE_INFINITY, 1L, NEGATIVE_INFINITY) diff --git a/presto-main/src/test/java/com/facebook/presto/operator/TestWindowOperator.java b/presto-main/src/test/java/com/facebook/presto/operator/TestWindowOperator.java index 2de50453c918..c141c7a5dc8a 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/TestWindowOperator.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/TestWindowOperator.java @@ -64,7 +64,7 @@ public class TestWindowOperator private static final FrameInfo UNBOUNDED_FRAME = new FrameInfo(RANGE, UNBOUNDED_PRECEDING, Optional.empty(), UNBOUNDED_FOLLOWING, Optional.empty()); private static final List ROW_NUMBER = ImmutableList.of( - window(new ReflectionWindowFunctionSupplier<>("row_number", BIGINT, ImmutableList.of(), RowNumberFunction.class), BIGINT, UNBOUNDED_FRAME) + window(new ReflectionWindowFunctionSupplier<>("row_number", BIGINT, ImmutableList.of(), RowNumberFunction.class), BIGINT, UNBOUNDED_FRAME) ); private static final List FIRST_VALUE = ImmutableList.of( diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/AbstractTestAggregationFunction.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/AbstractTestAggregationFunction.java index 84676f3fc69d..a0667b437544 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/AbstractTestAggregationFunction.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/AbstractTestAggregationFunction.java @@ -23,7 +23,6 @@ import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.TypeSignature; import com.facebook.presto.sql.analyzer.FeaturesConfig; -import com.facebook.presto.sql.analyzer.TypeSignatureProvider; import com.facebook.presto.sql.tree.QualifiedName; import com.facebook.presto.type.TypeRegistry; import com.google.common.collect.Lists; @@ -32,7 +31,6 @@ import java.util.List; import static com.facebook.presto.operator.aggregation.AggregationTestUtils.assertAggregation; -import static com.facebook.presto.sql.analyzer.TypeSignatureProvider.fromTypeSignatures; public abstract class AbstractTestAggregationFunction { @@ -43,8 +41,7 @@ public abstract class AbstractTestAggregationFunction protected final InternalAggregationFunction getFunction() { - List parameterTypes = fromTypeSignatures(Lists.transform(getFunctionParameterTypes(), TypeSignature::parseTypeSignature)); - Signature signature = functionRegistry.resolveFunction(QualifiedName.of(getFunctionName()), parameterTypes); + Signature signature = functionRegistry.resolveFunction(QualifiedName.of(getFunctionName()), Lists.transform(getFunctionParameterTypes(), TypeSignature::parseTypeSignature)); return functionRegistry.getAggregateFunctionImplementation(signature); } diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/AbstractTestDecimalAverageAggregation.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/AbstractTestDecimalAverageAggregation.java index 8bc12ec9f9f6..2b0b8eb1ed38 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/AbstractTestDecimalAverageAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/AbstractTestDecimalAverageAggregation.java @@ -42,7 +42,7 @@ public Block[] getSequenceBlocks(int start, int length) protected abstract void writeDecimalToBlock(BigDecimal decimal, BlockBuilder blockBuilder); - private static BigDecimal getBigDecimalForCounter(int i) + private BigDecimal getBigDecimalForCounter(int i) { String iAsString = String.valueOf(Math.abs(i)); return new BigDecimal(String.valueOf(i) + "." + iAsString + iAsString).setScale(2, ROUND_DOWN); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/AbstractTestDecimalSumAggregation.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/AbstractTestDecimalSumAggregation.java index e3ac72c68f3a..b5d3085208a8 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/AbstractTestDecimalSumAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/AbstractTestDecimalSumAggregation.java @@ -41,7 +41,7 @@ public Block[] getSequenceBlocks(int start, int length) protected abstract void writeDecimalToBlock(BigDecimal decimal, BlockBuilder blockBuilder); - private static BigDecimal getBigDecimalForCounter(int i) + private BigDecimal getBigDecimalForCounter(int i) { String iAsString = String.valueOf(Math.abs(i)); return new BigDecimal(String.valueOf(i) + "." + iAsString + iAsString).setScale(2, ROUND_DOWN); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestArrayMaxNAggregation.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestArrayMaxNAggregation.java index 8d76cd96ab4a..6f3859381f64 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestArrayMaxNAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestArrayMaxNAggregation.java @@ -36,7 +36,7 @@ public class TestArrayMaxNAggregation extends AbstractTestAggregationFunction { - public static Block createLongArraysBlock(Long[] values) + public Block createLongArraysBlock(Long[] values) { ArrayType arrayType = new ArrayType(BIGINT); BlockBuilder blockBuilder = arrayType.createBlockBuilder(new BlockBuilderStatus(), values.length); @@ -53,7 +53,7 @@ public static Block createLongArraysBlock(Long[] values) return blockBuilder.build(); } - public static Block createLongArraySequenceBlock(int start, int length) + public Block createLongArraySequenceBlock(int start, int length) { return createLongArraysBlock(LongStream.range(start, length).boxed().toArray(Long[]::new)); } diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestDoubleHistogramAggregation.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestDoubleHistogramAggregation.java index 1d013925d2a3..46aa7e948cb8 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestDoubleHistogramAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestDoubleHistogramAggregation.java @@ -130,7 +130,7 @@ private static Map extractSingleValue(Block block) return (Map) mapType.getObjectValue(null, block, 0); } - private static Page makeInput(int numberOfBuckets) + private Page makeInput(int numberOfBuckets) { PageBuilder builder = new PageBuilder(ImmutableList.of(BIGINT, DOUBLE, DOUBLE)); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestMinMaxByAggregation.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestMinMaxByAggregation.java index 1d51e32c3c57..5dbdca5c903f 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestMinMaxByAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestMinMaxByAggregation.java @@ -82,7 +82,7 @@ public void testAllRegistered() } } - private static List getTypes() + private List getTypes() { List simpleTypes = METADATA.getTypeManager().getTypes(); return new ImmutableList.Builder() diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestRealHistogramAggregation.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestRealHistogramAggregation.java index f234fc00c4a0..6b0ebaf4421e 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestRealHistogramAggregation.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestRealHistogramAggregation.java @@ -131,7 +131,7 @@ private static Map extractSingleValue(Block block) return (Map) mapType.getObjectValue(null, block, 0); } - private static Page makeInput(int numberOfBuckets) + private Page makeInput(int numberOfBuckets) { PageBuilder builder = new PageBuilder(ImmutableList.of(BIGINT, REAL, DOUBLE)); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestStateCompiler.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestStateCompiler.java index 2cd14c61afcd..0448928868ce 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestStateCompiler.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestStateCompiler.java @@ -35,8 +35,10 @@ public class TestStateCompiler @Test public void testPrimitiveNullableLongSerialization() { - AccumulatorStateFactory factory = StateCompiler.generateStateFactory(NullableLongState.class); - AccumulatorStateSerializer serializer = StateCompiler.generateStateSerializer(NullableLongState.class); + StateCompiler compiler = new StateCompiler(); + + AccumulatorStateFactory factory = compiler.generateStateFactory(NullableLongState.class); + AccumulatorStateSerializer serializer = compiler.generateStateSerializer(NullableLongState.class); NullableLongState state = factory.createSingleState(); NullableLongState deserializedState = factory.createSingleState(); @@ -61,8 +63,10 @@ public void testPrimitiveNullableLongSerialization() @Test public void testPrimitiveLongSerialization() { - AccumulatorStateFactory factory = StateCompiler.generateStateFactory(LongState.class); - AccumulatorStateSerializer serializer = StateCompiler.generateStateSerializer(LongState.class); + StateCompiler compiler = new StateCompiler(); + + AccumulatorStateFactory factory = compiler.generateStateFactory(LongState.class); + AccumulatorStateSerializer serializer = compiler.generateStateSerializer(LongState.class); LongState state = factory.createSingleState(); LongState deserializedState = factory.createSingleState(); @@ -81,15 +85,18 @@ public void testPrimitiveLongSerialization() @Test public void testGetSerializedType() { - AccumulatorStateSerializer serializer = StateCompiler.generateStateSerializer(LongState.class); + StateCompiler compiler = new StateCompiler(); + AccumulatorStateSerializer serializer = compiler.generateStateSerializer(LongState.class); assertEquals(serializer.getSerializedType(), BIGINT); } @Test public void testPrimitiveBooleanSerialization() { - AccumulatorStateFactory factory = StateCompiler.generateStateFactory(BooleanState.class); - AccumulatorStateSerializer serializer = StateCompiler.generateStateSerializer(BooleanState.class); + StateCompiler compiler = new StateCompiler(); + + AccumulatorStateFactory factory = compiler.generateStateFactory(BooleanState.class); + AccumulatorStateSerializer serializer = compiler.generateStateSerializer(BooleanState.class); BooleanState state = factory.createSingleState(); BooleanState deserializedState = factory.createSingleState(); @@ -106,8 +113,10 @@ public void testPrimitiveBooleanSerialization() @Test public void testPrimitiveByteSerialization() { - AccumulatorStateFactory factory = StateCompiler.generateStateFactory(ByteState.class); - AccumulatorStateSerializer serializer = StateCompiler.generateStateSerializer(ByteState.class); + StateCompiler compiler = new StateCompiler(); + + AccumulatorStateFactory factory = compiler.generateStateFactory(ByteState.class); + AccumulatorStateSerializer serializer = compiler.generateStateSerializer(ByteState.class); ByteState state = factory.createSingleState(); ByteState deserializedState = factory.createSingleState(); @@ -124,8 +133,10 @@ public void testPrimitiveByteSerialization() @Test public void testVarianceStateSerialization() { - AccumulatorStateFactory factory = StateCompiler.generateStateFactory(VarianceState.class); - AccumulatorStateSerializer serializer = StateCompiler.generateStateSerializer(VarianceState.class); + StateCompiler compiler = new StateCompiler(); + + AccumulatorStateFactory factory = compiler.generateStateFactory(VarianceState.class); + AccumulatorStateSerializer serializer = compiler.generateStateSerializer(VarianceState.class); VarianceState singleState = factory.createSingleState(); VarianceState deserializedState = factory.createSingleState(); @@ -147,8 +158,10 @@ public void testVarianceStateSerialization() @Test public void testComplexSerialization() { - AccumulatorStateFactory factory = StateCompiler.generateStateFactory(TestComplexState.class); - AccumulatorStateSerializer serializer = StateCompiler.generateStateSerializer(TestComplexState.class); + StateCompiler compiler = new StateCompiler(); + + AccumulatorStateFactory factory = compiler.generateStateFactory(TestComplexState.class); + AccumulatorStateSerializer serializer = compiler.generateStateSerializer(TestComplexState.class); TestComplexState singleState = factory.createSingleState(); TestComplexState deserializedState = factory.createSingleState(); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestTypedHeap.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestTypedHeap.java index 935ddcf5b4eb..a8b2b7bd375e 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestTypedHeap.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestTypedHeap.java @@ -70,7 +70,7 @@ public void testShuffled() IntStream.range(0, OUTPUT_SIZE).map(x -> OUTPUT_SIZE - 1 - x).iterator()); } - private static void test(IntStream inputStream, BlockComparator comparator, PrimitiveIterator.OfInt outputIterator) + private void test(IntStream inputStream, BlockComparator comparator, PrimitiveIterator.OfInt outputIterator) { BlockBuilder blockBuilder = BIGINT.createBlockBuilder(new BlockBuilderStatus(), INPUT_SIZE); inputStream.forEach(x -> BIGINT.writeLong(blockBuilder, x)); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestTypedKeyValueHeap.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestTypedKeyValueHeap.java index 48a9709b1d81..a3f9c240ceeb 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestTypedKeyValueHeap.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/TestTypedKeyValueHeap.java @@ -78,7 +78,7 @@ public void testShuffled() IntStream.range(0, OUTPUT_SIZE).map(x -> OUTPUT_SIZE - 1 - x).mapToObj(key -> Integer.toString(key * 2)).iterator()); } - private static void test(IntStream keyInputStream, Stream valueInputStream, BlockComparator comparator, Iterator outputIterator) + private void test(IntStream keyInputStream, Stream valueInputStream, BlockComparator comparator, Iterator outputIterator) { BlockBuilder keysBlockBuilder = BIGINT.createBlockBuilder(new BlockBuilderStatus(), INPUT_SIZE); BlockBuilder valuesBlockBuilder = VARCHAR.createBlockBuilder(new BlockBuilderStatus(), INPUT_SIZE); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/state/TestBigIntegerAndLongStateSerializer.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/state/TestBigIntegerAndLongStateSerializer.java index fd1df1975ec0..abf93a2495f5 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/state/TestBigIntegerAndLongStateSerializer.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/state/TestBigIntegerAndLongStateSerializer.java @@ -47,7 +47,7 @@ public void testRoundTrip() )); } - public static void testRoundTrip(List expected) + public void testRoundTrip(List expected) throws Exception { BigIntegerAndLongStateSerializer serializer = new BigIntegerAndLongStateSerializer(); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/state/TestBigIntegerStateSerializer.java b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/state/TestBigIntegerStateSerializer.java index 49f33cc5bc9b..d312a9819fa8 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/aggregation/state/TestBigIntegerStateSerializer.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/aggregation/state/TestBigIntegerStateSerializer.java @@ -45,7 +45,7 @@ public void testRoundTrip() )); } - private static void testRoundTrip(List expected) + private void testRoundTrip(List expected) { BigIntegerStateSerializer serializer = new BigIntegerStateSerializer(); BlockBuilder blockBuilder = new VariableWidthBlockBuilder(new BlockBuilderStatus(), expected.size(), 2); diff --git a/presto-main/src/test/java/com/facebook/presto/operator/index/TestTupleFilterProcessor.java b/presto-main/src/test/java/com/facebook/presto/operator/index/TestTupleFilterProcessor.java index 6544769ab086..6c0e5ff93d84 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/index/TestTupleFilterProcessor.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/index/TestTupleFilterProcessor.java @@ -41,7 +41,7 @@ public void testFullPageFilter() .row(1L, "a", 0.1) .build()); - List outputTypes = ImmutableList.of(VARCHAR, BIGINT, BOOLEAN, DOUBLE, DOUBLE); + List outputTypes = ImmutableList.of(VARCHAR, BIGINT, BOOLEAN, DOUBLE, DOUBLE); TupleFilterProcessor tupleFilterProcessor = new TupleFilterProcessor(tuplePage, outputTypes, new int[] { 1, 0, 3 }); Page inputPage = Iterables.getOnlyElement(rowPagesBuilder(outputTypes) @@ -72,7 +72,7 @@ public void testPartialPageFilter() .row(1L, "a", 0.1) .build()); - List outputTypes = ImmutableList.of(VARCHAR, BIGINT, BOOLEAN, DOUBLE, DOUBLE); + List outputTypes = ImmutableList.of(VARCHAR, BIGINT, BOOLEAN, DOUBLE, DOUBLE); TupleFilterProcessor tupleFilterProcessor = new TupleFilterProcessor(tuplePage, outputTypes, new int[] { 1, 0, 3 }); Page inputPage = Iterables.getOnlyElement(rowPagesBuilder(outputTypes) diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayFilter.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayFilter.java deleted file mode 100644 index 81f15b49a06a..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArrayFilter.java +++ /dev/null @@ -1,268 +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.operator.scalar; - -import com.facebook.presto.metadata.BoundVariables; -import com.facebook.presto.metadata.FunctionKind; -import com.facebook.presto.metadata.FunctionListBuilder; -import com.facebook.presto.metadata.FunctionRegistry; -import com.facebook.presto.metadata.MetadataManager; -import com.facebook.presto.metadata.Signature; -import com.facebook.presto.metadata.SqlScalarFunction; -import com.facebook.presto.operator.PageProcessor; -import com.facebook.presto.spi.Page; -import com.facebook.presto.spi.PageBuilder; -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.Type; -import com.facebook.presto.spi.type.TypeManager; -import com.facebook.presto.sql.gen.ExpressionCompiler; -import com.facebook.presto.sql.relational.CallExpression; -import com.facebook.presto.sql.relational.ConstantExpression; -import com.facebook.presto.sql.relational.InputReferenceExpression; -import com.facebook.presto.sql.relational.LambdaDefinitionExpression; -import com.facebook.presto.sql.relational.RowExpression; -import com.facebook.presto.sql.relational.VariableReferenceExpression; -import com.facebook.presto.type.ArrayType; -import com.google.common.base.Throwables; -import com.google.common.base.Verify; -import com.google.common.collect.ImmutableList; -import org.openjdk.jmh.annotations.Benchmark; -import org.openjdk.jmh.annotations.BenchmarkMode; -import org.openjdk.jmh.annotations.Fork; -import org.openjdk.jmh.annotations.Measurement; -import org.openjdk.jmh.annotations.Mode; -import org.openjdk.jmh.annotations.OperationsPerInvocation; -import org.openjdk.jmh.annotations.OutputTimeUnit; -import org.openjdk.jmh.annotations.Param; -import org.openjdk.jmh.annotations.Scope; -import org.openjdk.jmh.annotations.Setup; -import org.openjdk.jmh.annotations.State; -import org.openjdk.jmh.annotations.Warmup; -import org.openjdk.jmh.runner.Runner; -import org.openjdk.jmh.runner.options.Options; -import org.openjdk.jmh.runner.options.OptionsBuilder; -import org.openjdk.jmh.runner.options.VerboseMode; - -import java.lang.invoke.MethodHandle; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; - -import static com.facebook.presto.metadata.Signature.typeVariable; -import static com.facebook.presto.operator.scalar.BenchmarkArrayFilter.ExactArrayFilterFunction.EXACT_ARRAY_FILTER_FUNCTION; -import static com.facebook.presto.spi.function.OperatorType.GREATER_THAN; -import static com.facebook.presto.spi.type.BigintType.BIGINT; -import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; -import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature; -import static com.facebook.presto.spi.type.TypeUtils.readNativeValue; -import static com.facebook.presto.testing.TestingConnectorSession.SESSION; -import static com.facebook.presto.util.Reflection.methodHandle; -import static java.lang.Boolean.TRUE; - -@SuppressWarnings("MethodMayBeStatic") -@State(Scope.Thread) -@OutputTimeUnit(TimeUnit.NANOSECONDS) -@Fork(2) -@Warmup(iterations = 10, time = 500, timeUnit = TimeUnit.MILLISECONDS) -@Measurement(iterations = 10, time = 500, timeUnit = TimeUnit.MILLISECONDS) -@BenchmarkMode(Mode.AverageTime) -public class BenchmarkArrayFilter -{ - private static final int POSITIONS = 100_000; - private static final int ARRAY_SIZE = 4; - private static final int NUM_TYPES = 1; - private static final List TYPES = ImmutableList.of(BIGINT); - - static { - Verify.verify(NUM_TYPES == TYPES.size()); - } - - @Benchmark - @OperationsPerInvocation(POSITIONS * ARRAY_SIZE * NUM_TYPES) - public Object benchmark(BenchmarkData data) - throws Throwable - { - int position = 0; - List pages = new ArrayList<>(); - while (position < data.getPage().getPositionCount()) { - position = data.getPageProcessor().process(SESSION, data.getPage(), position, data.getPage().getPositionCount(), data.getPageBuilder()); - pages.add(data.getPageBuilder().build()); - data.getPageBuilder().reset(); - } - return pages; - } - - @SuppressWarnings("FieldMayBeFinal") - @State(Scope.Thread) - public static class BenchmarkData - { - @Param({"filter", "exact_filter"}) - private String name = "filter"; - - private PageBuilder pageBuilder; - private Page page; - private PageProcessor pageProcessor; - - @Setup - public void setup() - { - MetadataManager metadata = MetadataManager.createTestMetadataManager(); - metadata.addFunctions(new FunctionListBuilder().function(EXACT_ARRAY_FILTER_FUNCTION).getFunctions()); - ExpressionCompiler compiler = new ExpressionCompiler(metadata); - ImmutableList.Builder projectionsBuilder = ImmutableList.builder(); - Block[] blocks = new Block[TYPES.size()]; - for (int i = 0; i < TYPES.size(); i++) { - Type elementType = TYPES.get(i); - ArrayType arrayType = new ArrayType(elementType); - Signature signature = new Signature(name, FunctionKind.SCALAR, arrayType.getTypeSignature(), arrayType.getTypeSignature(), parseTypeSignature("function(bigint,boolean)")); - Signature greaterThan = new Signature("$operator$" + GREATER_THAN.name(), FunctionKind.SCALAR, BOOLEAN.getTypeSignature(), BIGINT.getTypeSignature(), BIGINT.getTypeSignature()); - projectionsBuilder.add(new CallExpression(signature, arrayType, ImmutableList.of( - new InputReferenceExpression(0, arrayType), - new LambdaDefinitionExpression( - ImmutableList.of(BIGINT), - ImmutableList.of("x"), - new CallExpression(greaterThan, BOOLEAN, ImmutableList.of(new VariableReferenceExpression("x", BIGINT), new ConstantExpression(0L, BIGINT))))))); - blocks[i] = createChannel(POSITIONS, ARRAY_SIZE, arrayType); - } - - ImmutableList projections = projectionsBuilder.build(); - pageProcessor = compiler.compilePageProcessor(new ConstantExpression(true, BOOLEAN), projections).get(); - pageBuilder = new PageBuilder(projections.stream().map(RowExpression::getType).collect(Collectors.toList())); - page = new Page(blocks); - } - - private static Block createChannel(int positionCount, int arraySize, ArrayType arrayType) - { - BlockBuilder blockBuilder = arrayType.createBlockBuilder(new BlockBuilderStatus(), positionCount); - for (int position = 0; position < positionCount; position++) { - BlockBuilder entryBuilder = blockBuilder.beginBlockEntry(); - for (int i = 0; i < arraySize; i++) { - if (arrayType.getElementType().getJavaType() == long.class) { - arrayType.getElementType().writeLong(entryBuilder, ThreadLocalRandom.current().nextLong()); - } - else { - throw new UnsupportedOperationException(); - } - } - blockBuilder.closeEntry(); - } - return blockBuilder.build(); - } - - public PageProcessor getPageProcessor() - { - return pageProcessor; - } - - public Page getPage() - { - return page; - } - - public PageBuilder getPageBuilder() - { - return pageBuilder; - } - } - - public static void main(String[] args) - throws Throwable - { - // assure the benchmarks are valid before running - BenchmarkData data = new BenchmarkData(); - data.setup(); - new BenchmarkArrayFilter().benchmark(data); - - Options options = new OptionsBuilder() - .verbosity(VerboseMode.NORMAL) - .include(".*" + BenchmarkArrayFilter.class.getSimpleName() + ".*") - .build(); - new Runner(options).run(); - } - - public static final class ExactArrayFilterFunction - extends SqlScalarFunction - { - public static final ExactArrayFilterFunction EXACT_ARRAY_FILTER_FUNCTION = new ExactArrayFilterFunction(); - - private static final MethodHandle METHOD_HANDLE = methodHandle(ExactArrayFilterFunction.class, "filter", Type.class, Block.class, MethodHandle.class); - - private ExactArrayFilterFunction() - { - super(new Signature( - "exact_filter", - FunctionKind.SCALAR, - ImmutableList.of(typeVariable("T")), - ImmutableList.of(), - parseTypeSignature("array(T)"), - ImmutableList.of(parseTypeSignature("array(T)"), parseTypeSignature("function(T,boolean)")), - false)); - } - - @Override - public boolean isHidden() - { - return false; - } - - @Override - public boolean isDeterministic() - { - return false; - } - - @Override - public String getDescription() - { - return "return array containing elements that match the given predicate"; - } - - @Override - public ScalarFunctionImplementation specialize(BoundVariables boundVariables, int arity, TypeManager typeManager, FunctionRegistry functionRegistry) - { - Type type = boundVariables.getTypeVariable("T"); - return new ScalarFunctionImplementation( - false, - ImmutableList.of(false, false), - METHOD_HANDLE.bindTo(type), - isDeterministic()); - } - - public static Block filter(Type type, Block block, MethodHandle function) - { - int positionCount = block.getPositionCount(); - BlockBuilder resultBuilder = type.createBlockBuilder(new BlockBuilderStatus(), positionCount); - for (int position = 0; position < positionCount; position++) { - Long input = (Long) readNativeValue(type, block, position); - Boolean keep; - try { - keep = (Boolean) function.invokeExact(input); - } - catch (Throwable throwable) { - throw Throwables.propagate(throwable); - } - if (TRUE.equals(keep)) { - block.writePositionTo(position, resultBuilder); - resultBuilder.closeEntry(); - } - } - return resultBuilder.build(); - } - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArraySubscript.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArraySubscript.java index a8cfa215a334..2eb72bc32e1f 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArraySubscript.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkArraySubscript.java @@ -176,12 +176,12 @@ public PageBuilder getPageBuilder() private static Block createArrayBlock(int positionCount, Block elementsBlock) { - int[] offsets = new int[positionCount + 1]; + int[] offsets = new int[positionCount]; int arraySize = elementsBlock.getPositionCount() / positionCount; - for (int i = 0; i < offsets.length; i++) { - offsets[i] = arraySize * i; + for (int i = 0; i < positionCount; i++) { + offsets[i] = arraySize * (i + 1); } - return new ArrayBlock(positionCount, new boolean[positionCount], offsets, elementsBlock); + return new ArrayBlock(elementsBlock, Slices.wrappedIntArray(offsets), 0, Slices.allocate(positionCount)); } private static Block createFixWidthValueBlock(int positionCount, int mapSize) diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkMapConcat.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkMapConcat.java deleted file mode 100644 index 87325a0b6f43..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkMapConcat.java +++ /dev/null @@ -1,236 +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.operator.scalar; - -import com.facebook.presto.metadata.FunctionKind; -import com.facebook.presto.metadata.MetadataManager; -import com.facebook.presto.metadata.Signature; -import com.facebook.presto.operator.PageProcessor; -import com.facebook.presto.spi.Page; -import com.facebook.presto.spi.PageBuilder; -import com.facebook.presto.spi.block.ArrayBlock; -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.DictionaryBlock; -import com.facebook.presto.spi.block.InterleavedBlock; -import com.facebook.presto.spi.block.SliceArrayBlock; -import com.facebook.presto.spi.type.BooleanType; -import com.facebook.presto.sql.gen.ExpressionCompiler; -import com.facebook.presto.sql.relational.CallExpression; -import com.facebook.presto.sql.relational.ConstantExpression; -import com.facebook.presto.sql.relational.InputReferenceExpression; -import com.facebook.presto.sql.relational.RowExpression; -import com.facebook.presto.type.MapType; -import com.google.common.collect.ImmutableList; -import io.airlift.slice.Slice; -import io.airlift.slice.Slices; -import org.openjdk.jmh.annotations.Benchmark; -import org.openjdk.jmh.annotations.BenchmarkMode; -import org.openjdk.jmh.annotations.Fork; -import org.openjdk.jmh.annotations.Measurement; -import org.openjdk.jmh.annotations.Mode; -import org.openjdk.jmh.annotations.OperationsPerInvocation; -import org.openjdk.jmh.annotations.OutputTimeUnit; -import org.openjdk.jmh.annotations.Param; -import org.openjdk.jmh.annotations.Scope; -import org.openjdk.jmh.annotations.Setup; -import org.openjdk.jmh.annotations.State; -import org.openjdk.jmh.annotations.Warmup; -import org.openjdk.jmh.runner.Runner; -import org.openjdk.jmh.runner.options.Options; -import org.openjdk.jmh.runner.options.OptionsBuilder; -import org.openjdk.jmh.runner.options.VerboseMode; -import org.openjdk.jmh.runner.options.WarmupMode; - -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; - -import static com.facebook.presto.spi.type.DoubleType.DOUBLE; -import static com.facebook.presto.spi.type.VarcharType.createUnboundedVarcharType; -import static com.facebook.presto.testing.TestingConnectorSession.SESSION; -import static io.airlift.slice.Slices.utf8Slice; - -@SuppressWarnings("MethodMayBeStatic") -@State(Scope.Thread) -@OutputTimeUnit(TimeUnit.NANOSECONDS) -@Fork(2) -@Warmup(iterations = 10, time = 500, timeUnit = TimeUnit.MILLISECONDS) -@Measurement(iterations = 10, time = 500, timeUnit = TimeUnit.MILLISECONDS) -@BenchmarkMode(Mode.AverageTime) -public class BenchmarkMapConcat -{ - private static final int POSITIONS = 10_000; - - @Benchmark - @OperationsPerInvocation(POSITIONS) - public Object mapConcat(BenchmarkData data) - throws Throwable - { - int position = 0; - List pages = new ArrayList<>(); - while (position < data.getPage().getPositionCount()) { - position = data.getPageProcessor().process(SESSION, data.getPage(), position, data.getPage().getPositionCount(), data.getPageBuilder()); - pages.add(data.getPageBuilder().build()); - data.getPageBuilder().reset(); - } - return pages; - } - - @SuppressWarnings("FieldMayBeFinal") - @State(Scope.Thread) - public static class BenchmarkData - { - private String name = "map_concat"; - - @Param({"left_empty", "right_empty", "both_empty", "non_empty"}) - private String mapConfig = "left_empty"; - - private PageBuilder pageBuilder; - private Page page; - private PageProcessor pageProcessor; - - @Setup - public void setup() - { - MetadataManager metadata = MetadataManager.createTestMetadataManager(); - ExpressionCompiler compiler = new ExpressionCompiler(metadata); - - List leftKeys; - List rightKeys; - switch (mapConfig) { - case "left_empty": - leftKeys = ImmutableList.of(); - rightKeys = ImmutableList.of("a", "b", "c"); - break; - case "right_empty": - leftKeys = ImmutableList.of("a", "b", "c"); - rightKeys = ImmutableList.of(); - break; - case "both_empty": - leftKeys = ImmutableList.of(); - rightKeys = ImmutableList.of(); - break; - case "non_empty": - leftKeys = ImmutableList.of("a", "b", "c"); - rightKeys = ImmutableList.of("d", "b", "c"); - break; - default: - throw new UnsupportedOperationException(); - } - - MapType mapType = new MapType(createUnboundedVarcharType(), DOUBLE); - - Block leftKeyBlock = createKeyBlock(POSITIONS, leftKeys); - Block leftValueBlock = createValueBlock(POSITIONS, leftKeys.size()); - Block leftBlock = createMapBlock(POSITIONS, leftKeyBlock, leftValueBlock); - - Block rightKeyBlock = createKeyBlock(POSITIONS, rightKeys); - Block rightValueBlock = createValueBlock(POSITIONS, rightKeys.size()); - Block rightBlock = createMapBlock(POSITIONS, rightKeyBlock, rightValueBlock); - - ImmutableList.Builder projectionsBuilder = ImmutableList.builder(); - - Signature signature = new Signature( - name, - FunctionKind.SCALAR, - mapType.getTypeSignature(), - mapType.getTypeSignature(), - mapType.getTypeSignature()); - projectionsBuilder.add(new CallExpression( - signature, - mapType, - ImmutableList.of(new InputReferenceExpression(0, mapType), new InputReferenceExpression(1, mapType)))); - - ImmutableList projections = projectionsBuilder.build(); - pageProcessor = compiler.compilePageProcessor(new ConstantExpression(true, BooleanType.BOOLEAN), projections).get(); - pageBuilder = new PageBuilder(projections.stream().map(RowExpression::getType).collect(Collectors.toList())); - page = new Page(leftBlock, rightBlock); - } - - public PageProcessor getPageProcessor() - { - return pageProcessor; - } - - public Page getPage() - { - return page; - } - - public PageBuilder getPageBuilder() - { - return pageBuilder; - } - - private static Block createMapBlock(int positionCount, Block keyBlock, Block valueBlock) - { - InterleavedBlock interleavedBlock = new InterleavedBlock(new Block[]{keyBlock, valueBlock}); - int[] offsets = new int[positionCount + 1]; - int mapSize = keyBlock.getPositionCount() / positionCount; - for (int i = 0; i < offsets.length; i++) { - offsets[i] = mapSize * 2 * i; - } - return new ArrayBlock(positionCount, new boolean[positionCount], offsets, interleavedBlock); - } - - private static Block createKeyBlock(int positionCount, List keys) - { - Block keyDictionaryBlock = createSliceArrayBlock(keys); - int[] keyIds = new int[positionCount * keys.size()]; - for (int i = 0; i < keyIds.length; i++) { - keyIds[i] = i % keys.size(); - } - return new DictionaryBlock(positionCount * keys.size(), keyDictionaryBlock, Slices.wrappedIntArray(keyIds)); - } - - private static Block createValueBlock(int positionCount, int mapSize) - { - BlockBuilder valueBlockBuilder = DOUBLE.createBlockBuilder(new BlockBuilderStatus(), positionCount * mapSize); - for (int i = 0; i < positionCount * mapSize; i++) { - DOUBLE.writeDouble(valueBlockBuilder, ThreadLocalRandom.current().nextDouble()); - } - return valueBlockBuilder.build(); - } - - private static Block createSliceArrayBlock(List keys) - { - // last position is reserved for null - Slice[] sliceArray = new Slice[keys.size() + 1]; - for (int i = 0; i < keys.size(); i++) { - sliceArray[i] = utf8Slice(keys.get(i)); - } - return new SliceArrayBlock(sliceArray.length, sliceArray); - } - } - - public static void main(String[] args) - throws Throwable - { - // assure the benchmarks are valid before running - BenchmarkData data = new BenchmarkData(); - data.setup(); - new BenchmarkMapConcat().mapConcat(data); - - Options options = new OptionsBuilder() - .verbosity(VerboseMode.NORMAL) - .warmupMode(WarmupMode.INDI) - .include(".*" + BenchmarkMapConcat.class.getSimpleName() + ".*") - .build(); - new Runner(options).run(); - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkMapSubscript.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkMapSubscript.java index b6d56e40d517..f56a5959d162 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkMapSubscript.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/BenchmarkMapSubscript.java @@ -189,12 +189,12 @@ public PageBuilder getPageBuilder() private static Block createMapBlock(int positionCount, Block keyBlock, Block valueBlock) { InterleavedBlock interleavedBlock = new InterleavedBlock(new Block[] {keyBlock, valueBlock}); - int[] offsets = new int[positionCount + 1]; + int[] offsets = new int[positionCount]; int mapSize = keyBlock.getPositionCount() / positionCount; - for (int i = 0; i < offsets.length; i++) { + for (int i = 0; i < positionCount; i++) { offsets[i] = mapSize * 2 * i; } - return new ArrayBlock(positionCount, new boolean[positionCount], offsets, interleavedBlock); + return new ArrayBlock(interleavedBlock, Slices.wrappedIntArray(offsets), 0, Slices.allocate(positionCount)); } private static Block createKeyBlock(int positionCount, List keys) diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/FunctionAssertions.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/FunctionAssertions.java index 4c34e7a50491..983c3ea39cc6 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/FunctionAssertions.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/FunctionAssertions.java @@ -582,14 +582,14 @@ private Operator interpretedFilterProject(Expression filter, Expression projecti private OperatorFactory compileFilterWithNoInputColumns(Expression filter, ExpressionCompiler compiler) { - filter = new SymbolToInputRewriter(ImmutableMap.of()).rewrite(filter); + filter = ExpressionTreeRewriter.rewriteWith(new SymbolToInputRewriter(ImmutableMap.of()), filter); IdentityHashMap expressionTypes = getExpressionTypesFromInput(TEST_SESSION, metadata, SQL_PARSER, INPUT_TYPES, ImmutableList.of(filter), emptyList()); try { Supplier processor = compiler.compilePageProcessor(toRowExpression(filter, expressionTypes), ImmutableList.of()); - return new FilterAndProjectOperator.FilterAndProjectOperatorFactory(0, new PlanNodeId("test"), processor, ImmutableList.of()); + return new FilterAndProjectOperator.FilterAndProjectOperatorFactory(0, new PlanNodeId("test"), processor, ImmutableList.of()); } catch (Throwable e) { if (e instanceof UncheckedExecutionException) { @@ -601,8 +601,8 @@ private OperatorFactory compileFilterWithNoInputColumns(Expression filter, Expre private OperatorFactory compileFilterProject(Expression filter, Expression projection, ExpressionCompiler compiler) { - filter = new SymbolToInputRewriter(INPUT_MAPPING).rewrite(filter); - projection = new SymbolToInputRewriter(INPUT_MAPPING).rewrite(projection); + filter = ExpressionTreeRewriter.rewriteWith(new SymbolToInputRewriter(INPUT_MAPPING), filter); + projection = ExpressionTreeRewriter.rewriteWith(new SymbolToInputRewriter(INPUT_MAPPING), projection); IdentityHashMap expressionTypes = getExpressionTypesFromInput(TEST_SESSION, metadata, SQL_PARSER, INPUT_TYPES, ImmutableList.of(filter, projection), emptyList()); @@ -623,8 +623,8 @@ private OperatorFactory compileFilterProject(Expression filter, Expression proje private SourceOperatorFactory compileScanFilterProject(Expression filter, Expression projection, ExpressionCompiler compiler) { - filter = new SymbolToInputRewriter(INPUT_MAPPING).rewrite(filter); - projection = new SymbolToInputRewriter(INPUT_MAPPING).rewrite(projection); + filter = ExpressionTreeRewriter.rewriteWith(new SymbolToInputRewriter(INPUT_MAPPING), filter); + projection = ExpressionTreeRewriter.rewriteWith(new SymbolToInputRewriter(INPUT_MAPPING), projection); IdentityHashMap expressionTypes = getExpressionTypesFromInput(TEST_SESSION, metadata, SQL_PARSER, INPUT_TYPES, ImmutableList.of(filter, projection), emptyList()); @@ -646,7 +646,7 @@ private SourceOperatorFactory compileScanFilterProject(Expression filter, Expres PAGE_SOURCE_PROVIDER, cursorProcessor, pageProcessor, - ImmutableList.of(), + ImmutableList.of(), ImmutableList.of(expressionTypes.get(projection))); } catch (Throwable e) { @@ -713,7 +713,7 @@ public ConnectorPageSource createPageSource(Session session, Split split, Listof(BIGINT, VARCHAR, DOUBLE, BOOLEAN, BIGINT, VARCHAR, VARCHAR, TIMESTAMP_WITH_TIME_ZONE, VARBINARY, INTEGER)) .addRow( 1234L, "hello", diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestApplyFunction.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestApplyFunction.java deleted file mode 100644 index 40573594b38d..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestApplyFunction.java +++ /dev/null @@ -1,155 +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.operator.scalar; - -import com.facebook.presto.Session; -import com.facebook.presto.type.ArrayType; -import com.facebook.presto.type.MapType; -import com.facebook.presto.type.RowType; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import org.testng.annotations.Test; - -import java.util.Optional; - -import static com.facebook.presto.operator.scalar.ApplyFunction.APPLY_FUNCTION; -import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; -import static com.facebook.presto.spi.type.BigintType.BIGINT; -import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; -import static com.facebook.presto.spi.type.DoubleType.DOUBLE; -import static com.facebook.presto.spi.type.IntegerType.INTEGER; -import static com.facebook.presto.spi.type.TimeZoneKey.getTimeZoneKey; -import static com.facebook.presto.spi.type.VarcharType.VARCHAR; -import static com.facebook.presto.spi.type.VarcharType.createUnboundedVarcharType; -import static com.facebook.presto.spi.type.VarcharType.createVarcharType; -import static com.facebook.presto.testing.TestingSession.testSessionBuilder; - -public class TestApplyFunction - extends AbstractTestFunctions -{ - public TestApplyFunction() - { - this(testSessionBuilder().setTimeZoneKey(getTimeZoneKey("Pacific/Kiritimati")).build()); - } - - private TestApplyFunction(Session session) - { - super(session); - functionAssertions.getMetadata().addFunctions(ImmutableList.of(APPLY_FUNCTION)); - } - - @Test - public void testBasic() - throws Exception - { - assertFunction("apply(5, x -> x + 1)", INTEGER, 6); - assertFunction("apply(5 + RANDOM(1), x -> x + 1)", INTEGER, 6); - } - - @Test - public void testNull() - throws Exception - { - assertFunction("apply(3, x -> x + 1)", INTEGER, 4); - assertFunction("apply(NULL, x -> x + 1)", INTEGER, null); - assertFunction("apply(CAST (NULL AS INTEGER), x -> x + 1)", INTEGER, null); - - assertFunction("apply(3, x -> x IS NULL)", BOOLEAN, false); - assertFunction("apply(NULL, x -> x IS NULL)", BOOLEAN, true); - assertFunction("apply(CAST (NULL AS INTEGER), x -> x IS NULL)", BOOLEAN, true); - } - - @Test - public void testUnreferencedLambdaArgument() - { - assertFunction("apply(5, x -> 6)", INTEGER, 6); - } - - @Test - public void testSessionDependent() - throws Exception - { - assertFunction("apply('timezone: ', x -> x || current_timezone())", VARCHAR, "timezone: Pacific/Kiritimati"); - } - - @Test - public void testInstanceFunction() - { - assertFunction("apply(ARRAY[2], x -> concat(ARRAY [1], x))", new ArrayType(INTEGER), ImmutableList.of(1, 2)); - } - - @Test - public void testWithTry() - throws Exception - { - assertFunction("TRY(apply(5, x -> x + 1) / 0)", INTEGER, null); - assertFunction("TRY(apply(5 + RANDOM(1), x -> x + 1) / 0)", INTEGER, null); - assertInvalidFunction("apply(5 + RANDOM(1), x -> x + TRY(1 / 0))", NOT_SUPPORTED); - } - - @Test - public void testNestedLambda() - throws Exception - { - assertFunction("apply(11, x -> apply(x + 7, y -> apply(y * 3, z -> z * 5) + 1) * 2)", INTEGER, 542); - assertFunction("apply(11, x -> apply(x + 7, x -> apply(x * 3, x -> x * 5) + 1) * 2)", INTEGER, 542); - } - - @Test - public void testRowAccess() - throws Exception - { - assertFunction("apply(CAST(ROW(1, 'a') AS ROW(x INTEGER, y VARCHAR)), r -> r.x)", INTEGER, 1); - assertFunction("apply(CAST(ROW(1, 'a') AS ROW(x INTEGER, y VARCHAR)), r -> r.y)", VARCHAR, "a"); - } - - @Test - public void testTypeCombinations() - throws Exception - { - assertFunction("apply(25, x -> x + 1)", INTEGER, 26); - assertFunction("apply(25, x -> x + 1.0)", DOUBLE, 26.0); - assertFunction("apply(25, x -> x = 25)", BOOLEAN, true); - assertFunction("apply(25, x -> to_base(x, 16))", createUnboundedVarcharType(), "19"); - assertFunction("apply(25, x -> ARRAY[x + 1])", new ArrayType(INTEGER), ImmutableList.of(26)); - - assertFunction("apply(25.6, x -> CAST(x AS BIGINT))", BIGINT, 26L); - assertFunction("apply(25.6, x -> x + 1.0)", DOUBLE, 26.6); - assertFunction("apply(25.6, x -> x = 25.6)", BOOLEAN, true); - assertFunction("apply(25.6, x -> CAST(x AS VARCHAR))", createUnboundedVarcharType(), "25.6"); - assertFunction("apply(25.6, x -> MAP(ARRAY[x + 1], ARRAY[true]))", new MapType(DOUBLE, BOOLEAN), ImmutableMap.of(26.6, true)); - - assertFunction("apply(true, x -> if(x, 25, 26))", INTEGER, 25); - assertFunction("apply(false, x -> if(x, 25.6, 28.9))", DOUBLE, 28.9); - assertFunction("apply(true, x -> not x)", BOOLEAN, false); - assertFunction("apply(false, x -> CAST(x AS VARCHAR))", createUnboundedVarcharType(), "false"); - assertFunction("apply(true, x -> ARRAY[x])", new ArrayType(BOOLEAN), ImmutableList.of(true)); - - assertFunction("apply('41', x -> from_base(x, 16))", BIGINT, 65L); - assertFunction("apply('25.6', x -> CAST(x AS DOUBLE))", DOUBLE, 25.6); - assertFunction("apply('abc', x -> 'abc' = x)", BOOLEAN, true); - assertFunction("apply('abc', x -> x || x)", createUnboundedVarcharType(), "abcabc"); - assertFunction( - "apply('123', x -> ROW(x, CAST(x AS INTEGER), x > '0'))", - new RowType(ImmutableList.of(createVarcharType(3), INTEGER, BOOLEAN), Optional.empty()), - ImmutableList.of("123", 123, true)); - - assertFunction("apply(ARRAY['abc', NULL, '123'], x -> from_base(x[3], 10))", BIGINT, 123L); - assertFunction("apply(ARRAY['abc', NULL, '123'], x -> CAST(x[3] AS DOUBLE))", DOUBLE, 123.0); - assertFunction("apply(ARRAY['abc', NULL, '123'], x -> x[2] IS NULL)", BOOLEAN, true); - assertFunction("apply(ARRAY['abc', NULL, '123'], x -> x[2])", createVarcharType(3), null); - assertFunction("apply(MAP(ARRAY['abc', 'def'], ARRAY[123, 456]), x -> map_keys(x))", new ArrayType(createVarcharType(3)), ImmutableList.of("abc", "def")); - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestArrayFilterFunction.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestArrayFilterFunction.java deleted file mode 100644 index 08b65c5a3e0a..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestArrayFilterFunction.java +++ /dev/null @@ -1,76 +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.operator.scalar; - -import com.facebook.presto.type.ArrayType; -import com.google.common.collect.ImmutableList; -import org.testng.annotations.Test; - -import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; -import static com.facebook.presto.spi.type.DoubleType.DOUBLE; -import static com.facebook.presto.spi.type.IntegerType.INTEGER; -import static com.facebook.presto.spi.type.VarcharType.createVarcharType; -import static com.facebook.presto.type.UnknownType.UNKNOWN; -import static java.util.Arrays.asList; -import static java.util.Collections.singletonList; - -public class TestArrayFilterFunction - extends AbstractTestFunctions -{ - @Test - public void testBasic() - throws Exception - { - assertFunction("filter(ARRAY [5, 6], x -> x = 5)", new ArrayType(INTEGER), ImmutableList.of(5)); - assertFunction("filter(ARRAY [5 + RANDOM(1), 6 + RANDOM(1)], x -> x = 5)", new ArrayType(INTEGER), ImmutableList.of(5)); - assertFunction("filter(ARRAY [true, false, true, false], x -> nullif(x, false))", new ArrayType(BOOLEAN), ImmutableList.of(true, true)); - assertFunction("filter(ARRAY [true, false, null, true, false, null], x -> not x)", new ArrayType(BOOLEAN), ImmutableList.of(false, false)); - } - - @Test - public void testEmpty() - throws Exception - { - assertFunction("filter(ARRAY [], x -> true)", new ArrayType(UNKNOWN), ImmutableList.of()); - assertFunction("filter(ARRAY [], x -> false)", new ArrayType(UNKNOWN), ImmutableList.of()); - assertFunction("filter(ARRAY [], x -> CAST (null AS BOOLEAN))", new ArrayType(UNKNOWN), ImmutableList.of()); - assertFunction("filter(CAST (ARRAY [] AS ARRAY(INTEGER)), x -> true)", new ArrayType(INTEGER), ImmutableList.of()); - } - - @Test - public void testNull() - throws Exception - { - assertFunction("filter(ARRAY [NULL], x -> x IS NULL)", new ArrayType(UNKNOWN), singletonList(null)); - assertFunction("filter(ARRAY [NULL], x -> x IS NOT NULL)", new ArrayType(UNKNOWN), ImmutableList.of()); - assertFunction("filter(ARRAY [CAST (NULL AS INTEGER)], x -> x IS NULL)", new ArrayType(INTEGER), singletonList(null)); - assertFunction("filter(ARRAY [NULL, NULL, NULL], x -> x IS NULL)", new ArrayType(UNKNOWN), asList(null, null, null)); - assertFunction("filter(ARRAY [NULL, NULL, NULL], x -> x IS NOT NULL)", new ArrayType(UNKNOWN), ImmutableList.of()); - } - - @Test - public void testTypeCombinations() - throws Exception - { - assertFunction("filter(ARRAY [25, 26, 27], x -> x % 2 = 1)", new ArrayType(INTEGER), ImmutableList.of(25, 27)); - assertFunction("filter(ARRAY [25.6, 37.3, 28.6], x -> x < 30.0)", new ArrayType(DOUBLE), ImmutableList.of(25.6, 28.6)); - assertFunction("filter(ARRAY [true, false, true], x -> not x)", new ArrayType(BOOLEAN), ImmutableList.of(false)); - assertFunction("filter(ARRAY ['abc', 'def', 'ayz'], x -> substr(x, 1, 1) = 'a')", new ArrayType(createVarcharType(3)), ImmutableList.of("abc", "ayz")); - assertFunction( - "filter(ARRAY [ARRAY ['abc', null, '123'], ARRAY ['def', 'x', '456']], x -> x[2] IS NULL)", - new ArrayType(new ArrayType(createVarcharType(3))), - ImmutableList.of(asList("abc", null, "123"))); - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestArrayReduceFunction.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestArrayReduceFunction.java deleted file mode 100644 index ea67f8d71983..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestArrayReduceFunction.java +++ /dev/null @@ -1,98 +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.operator.scalar; - -import com.facebook.presto.type.ArrayType; -import org.testng.annotations.Test; - -import static com.facebook.presto.spi.type.BigintType.BIGINT; -import static com.facebook.presto.spi.type.DoubleType.DOUBLE; -import static com.facebook.presto.spi.type.IntegerType.INTEGER; -import static com.facebook.presto.spi.type.TimeZoneKey.getTimeZoneKey; -import static com.facebook.presto.testing.TestingSession.testSessionBuilder; -import static java.util.Arrays.asList; - -public class TestArrayReduceFunction - extends AbstractTestFunctions -{ - public TestArrayReduceFunction() - { - super(testSessionBuilder().setTimeZoneKey(getTimeZoneKey("Pacific/Kiritimati")).build()); - } - - @Test - public void testEmpty() - throws Exception - { - assertFunction("reduce(ARRAY [], CAST (0 AS BIGINT), (s, x) -> s + x, s -> s)", BIGINT, 0L); - } - - @Test - public void testBasic() - throws Exception - { - assertFunction("reduce(ARRAY [5, 20, 50], CAST (0 AS BIGINT), (s, x) -> s + x, s -> s)", BIGINT, 75L); - assertFunction("reduce(ARRAY [5 + RANDOM(1), 20, 50], CAST (0 AS BIGINT), (s, x) -> s + x, s -> s)", BIGINT, 75L); - assertFunction("reduce(ARRAY [5, 6, 10, 20], 0.0, (s, x) -> s + x, s -> s)", DOUBLE, 41.0); - } - - @Test - public void testNulls() - throws Exception - { - assertFunction("reduce(ARRAY [NULL], CAST (0 AS BIGINT), (s, x) -> s + x, s -> s)", BIGINT, null); - assertFunction("reduce(ARRAY [NULL, NULL, NULL], CAST (0 AS BIGINT), (s, x) -> coalesce(x, 1) + s, s -> s)", BIGINT, 3L); - assertFunction("reduce(ARRAY [5, NULL, 50], CAST (0 AS BIGINT), (s, x) -> s + x, s -> s)", BIGINT, null); - assertFunction("reduce(ARRAY [5, NULL, 50], CAST (0 AS BIGINT), (s, x) -> coalesce(x, 0) + s, s -> s)", BIGINT, 55L); - - // mimics max function - assertFunction("reduce(ARRAY [], CAST (NULL AS BIGINT), (s, x) -> IF(s IS NULL OR x > s, x, s), s -> s)", BIGINT, null); - assertFunction("reduce(ARRAY [NULL], CAST (NULL AS BIGINT), (s, x) -> IF(s IS NULL OR x > s, x, s), s -> s)", BIGINT, null); - assertFunction("reduce(ARRAY [NULL, NULL, NULL], CAST (NULL AS BIGINT), (s, x) -> IF(s IS NULL OR x > s, x, s), s -> s)", BIGINT, null); - assertFunction("reduce(ARRAY [NULL, 6, 10, NULL, 8], CAST (NULL AS BIGINT), (s, x) -> IF(s IS NULL OR x > s, x, s), s -> s)", BIGINT, 10L); - assertFunction("reduce(ARRAY [5, NULL, 6, 10, NULL, 8], CAST (NULL AS BIGINT), (s, x) -> IF(s IS NULL OR x > s, x, s), s -> s)", BIGINT, 10L); - } - - @Test - public void testTwoValueState() - throws Exception - { - assertFunction( - "reduce(" + - "ARRAY [5, 20, 50], " + - "CAST(ROW(0, 0) AS ROW(sum BIGINT, count INTEGER)), " + - "(s, x) -> CAST(ROW(x + s.sum, s.count + 1) AS ROW(sum BIGINT, count INTEGER)), " + - "s -> s.sum / s.count)", - BIGINT, - 25L); - assertFunction( - "reduce(" + - "ARRAY [5, 6, 10, 20], " + - "CAST(ROW(0.0, 0) AS ROW(sum DOUBLE, count INTEGER)), " + - "(s, x) -> CAST(ROW(x + s.sum, s.count + 1) AS ROW(sum DOUBLE, count INTEGER)), " + - "s -> s.sum / s.count)", - DOUBLE, - 10.25); - } - - @Test - public void testInstanceFunction() - { - assertFunction( - "reduce(ARRAY[ARRAY[1, 2], ARRAY[3, 4], ARRAY[5, NULL, 7]], CAST(ARRAY[] AS ARRAY(INTEGER)), (s, x) -> concat(s, x), s -> s)", - new ArrayType(INTEGER), - asList(1, 2, 3, 4, 5, null, 7)); - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestArrayTransformFunction.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestArrayTransformFunction.java deleted file mode 100644 index ef785a692799..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestArrayTransformFunction.java +++ /dev/null @@ -1,135 +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.operator.scalar; - -import com.facebook.presto.type.ArrayType; -import com.facebook.presto.type.MapType; -import com.facebook.presto.type.RowType; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import org.testng.annotations.Test; - -import java.util.Optional; - -import static com.facebook.presto.spi.type.BigintType.BIGINT; -import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; -import static com.facebook.presto.spi.type.DoubleType.DOUBLE; -import static com.facebook.presto.spi.type.IntegerType.INTEGER; -import static com.facebook.presto.spi.type.TimeZoneKey.getTimeZoneKey; -import static com.facebook.presto.spi.type.VarcharType.VARCHAR; -import static com.facebook.presto.spi.type.VarcharType.createUnboundedVarcharType; -import static com.facebook.presto.spi.type.VarcharType.createVarcharType; -import static com.facebook.presto.testing.TestingSession.testSessionBuilder; -import static java.util.Arrays.asList; - -public class TestArrayTransformFunction - extends AbstractTestFunctions -{ - public TestArrayTransformFunction() - { - super(testSessionBuilder().setTimeZoneKey(getTimeZoneKey("Pacific/Kiritimati")).build()); - } - - @Test - public void testBasic() - throws Exception - { - assertFunction("transform(ARRAY [5, 6], x -> x + 1)", new ArrayType(INTEGER), ImmutableList.of(6, 7)); - assertFunction("transform(ARRAY [5 + RANDOM(1), 6], x -> x + 1)", new ArrayType(INTEGER), ImmutableList.of(6, 7)); - } - - @Test - public void testNull() - throws Exception - { - assertFunction("transform(ARRAY [3], x -> x + 1)", new ArrayType(INTEGER), ImmutableList.of(4)); - assertFunction("transform(ARRAY [NULL, NULL], x -> x + 1)", new ArrayType(INTEGER), asList(null, null)); - assertFunction("transform(ARRAY [NULL, 3, NULL], x -> x + 1)", new ArrayType(INTEGER), asList(null, 4, null)); - - assertFunction("transform(ARRAY [3], x -> x IS NULL)", new ArrayType(BOOLEAN), ImmutableList.of(false)); - assertFunction("transform(ARRAY [NULL, NULL], x -> x IS NULL)", new ArrayType(BOOLEAN), ImmutableList.of(true, true)); - assertFunction("transform(ARRAY [NULL, 3, NULL], x -> x IS NULL)", new ArrayType(BOOLEAN), ImmutableList.of(true, false, true)); - } - - @Test - public void testSessionDependent() - throws Exception - { - assertFunction("transform(ARRAY['timezone: ', 'tz: '], x -> x || current_timezone())", new ArrayType(VARCHAR), ImmutableList.of("timezone: Pacific/Kiritimati", "tz: Pacific/Kiritimati")); - } - - @Test - public void testInstanceFunction() - { - assertFunction("transform(ARRAY[2, 3, 4, NULL, 5], x -> concat(ARRAY [1], x))", new ArrayType(new ArrayType(INTEGER)), - asList(ImmutableList.of(1, 2), ImmutableList.of(1, 3), ImmutableList.of(1, 4), null, ImmutableList.of(1, 5))); - } - - @Test - public void testTypeCombinations() - throws Exception - { - assertFunction("transform(ARRAY [25, 26], x -> x + 1)", new ArrayType(INTEGER), ImmutableList.of(26, 27)); - assertFunction("transform(ARRAY [25, 26], x -> x + 1.0)", new ArrayType(DOUBLE), ImmutableList.of(26.0, 27.0)); - assertFunction("transform(ARRAY [25, 26], x -> x = 25)", new ArrayType(BOOLEAN), ImmutableList.of(true, false)); - assertFunction("transform(ARRAY [25, 26], x -> to_base(x, 16))", new ArrayType(createUnboundedVarcharType()), ImmutableList.of("19", "1a")); - assertFunction("transform(ARRAY [25, 26], x -> ARRAY[x + 1])", new ArrayType(new ArrayType(INTEGER)), ImmutableList.of(ImmutableList.of(26), ImmutableList.of(27))); - - assertFunction("transform(ARRAY [25.6, 27.3], x -> CAST(x AS BIGINT))", new ArrayType(BIGINT), ImmutableList.of(26L, 27L)); - assertFunction("transform(ARRAY [25.6, 27.3], x -> x + 1.0)", new ArrayType(DOUBLE), ImmutableList.of(26.6, 28.3)); - assertFunction("transform(ARRAY [25.6, 27.3], x -> x = 25.6)", new ArrayType(BOOLEAN), ImmutableList.of(true, false)); - assertFunction("transform(ARRAY [25.6, 27.3], x -> CAST(x AS VARCHAR))", new ArrayType(createUnboundedVarcharType()), ImmutableList.of("25.6", "27.3")); - assertFunction( - "transform(ARRAY [25.6, 27.3], x -> MAP(ARRAY[x + 1], ARRAY[true]))", - new ArrayType(new MapType(DOUBLE, BOOLEAN)), - ImmutableList.of(ImmutableMap.of(26.6, true), ImmutableMap.of(28.3, true))); - - assertFunction("transform(ARRAY [true, false], x -> if(x, 25, 26))", new ArrayType(INTEGER), ImmutableList.of(25, 26)); - assertFunction("transform(ARRAY [false, true], x -> if(x, 25.6, 28.9))", new ArrayType(DOUBLE), ImmutableList.of(28.9, 25.6)); - assertFunction("transform(ARRAY [true, false], x -> not x)", new ArrayType(BOOLEAN), ImmutableList.of(false, true)); - assertFunction("transform(ARRAY [false, true], x -> CAST(x AS VARCHAR))", new ArrayType(createUnboundedVarcharType()), ImmutableList.of("false", "true")); - assertFunction("transform(ARRAY [true, false], x -> ARRAY[x])", new ArrayType(new ArrayType(BOOLEAN)), ImmutableList.of(ImmutableList.of(true), ImmutableList.of(false))); - - assertFunction("transform(ARRAY ['41', '42'], x -> from_base(x, 16))", new ArrayType(BIGINT), ImmutableList.of(65L, 66L)); - assertFunction("transform(ARRAY ['25.6', '27.3'], x -> CAST(x AS DOUBLE))", new ArrayType(DOUBLE), ImmutableList.of(25.6, 27.3)); - assertFunction("transform(ARRAY ['abc', 'def'], x -> 'abc' = x)", new ArrayType(BOOLEAN), ImmutableList.of(true, false)); - assertFunction("transform(ARRAY ['abc', 'def'], x -> x || x)", new ArrayType(createUnboundedVarcharType()), ImmutableList.of("abcabc", "defdef")); - assertFunction( - "transform(ARRAY ['123', '456'], x -> ROW(x, CAST(x AS INTEGER), x > '3'))", - new ArrayType(new RowType(ImmutableList.of(createVarcharType(3), INTEGER, BOOLEAN), Optional.empty())), - ImmutableList.of(ImmutableList.of("123", 123, false), ImmutableList.of("456", 456, true))); - - assertFunction( - "transform(ARRAY [ARRAY ['abc', null, '123'], ARRAY ['def', 'x', '456']], x -> from_base(x[3], 10))", - new ArrayType(BIGINT), - ImmutableList.of(123L, 456L)); - assertFunction( - "transform(ARRAY [ARRAY ['abc', null, '123'], ARRAY ['def', 'x', '456']], x -> CAST(x[3] AS DOUBLE))", - new ArrayType(DOUBLE), - ImmutableList.of(123.0, 456.0)); - assertFunction( - "transform(ARRAY [ARRAY ['abc', null, '123'], ARRAY ['def', 'x', '456']], x -> x[2] IS NULL)", - new ArrayType(BOOLEAN), - ImmutableList.of(true, false)); - assertFunction( - "transform(ARRAY [ARRAY ['abc', null, '123'], ARRAY ['def', 'x', '456']], x -> x[2])", - new ArrayType(createVarcharType(3)), - asList(null, "x")); - assertFunction( - "transform(ARRAY [MAP(ARRAY['abc', 'def'], ARRAY[123, 456]), MAP(ARRAY['ghi', 'jkl'], ARRAY[234, 567])], x -> map_keys(x))", - new ArrayType(new ArrayType(createVarcharType(3))), - ImmutableList.of(ImmutableList.of("abc", "def"), ImmutableList.of("ghi", "jkl"))); - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestMapFilterFunction.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestMapFilterFunction.java deleted file mode 100644 index 81e3d531be8c..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestMapFilterFunction.java +++ /dev/null @@ -1,197 +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.operator.scalar; - -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 org.testng.annotations.Test; - -import java.util.HashMap; -import java.util.Map; - -import static com.facebook.presto.spi.type.BigintType.BIGINT; -import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; -import static com.facebook.presto.spi.type.DoubleType.DOUBLE; -import static com.facebook.presto.spi.type.IntegerType.INTEGER; -import static com.facebook.presto.spi.type.VarcharType.VARCHAR; -import static com.facebook.presto.spi.type.VarcharType.createVarcharType; -import static com.facebook.presto.type.UnknownType.UNKNOWN; - -public class TestMapFilterFunction - extends AbstractTestFunctions -{ - @Test - public void testEmpty() - throws Exception - { - assertFunction("map_filter(map(ARRAY[], ARRAY[]), (k, v) -> true)", new MapType(UNKNOWN, UNKNOWN), ImmutableMap.of()); - assertFunction("map_filter(map(ARRAY[], ARRAY[]), (k, v) -> false)", new MapType(UNKNOWN, UNKNOWN), ImmutableMap.of()); - assertFunction("map_filter(map(ARRAY[], ARRAY[]), (k, v) -> CAST (NULL AS BOOLEAN))", new MapType(UNKNOWN, UNKNOWN), ImmutableMap.of()); - assertFunction("map_filter(CAST (map(ARRAY[], ARRAY[]) AS MAP(BIGINT,VARCHAR)), (k, v) -> true)", new MapType(BIGINT, VARCHAR), ImmutableMap.of()); - } - - @Test - public void testNull() - throws Exception - { - Map oneToNullMap = new HashMap<>(); - oneToNullMap.put(1, null); - assertFunction("map_filter(map(ARRAY[1], ARRAY [NULL]), (k, v) -> v IS NULL)", new MapType(INTEGER, UNKNOWN), oneToNullMap); - assertFunction("map_filter(map(ARRAY[1], ARRAY [NULL]), (k, v) -> v IS NOT NULL)", new MapType(INTEGER, UNKNOWN), ImmutableMap.of()); - assertFunction("map_filter(map(ARRAY[1], ARRAY [CAST (NULL AS INTEGER)]), (k, v) -> v IS NULL)", new MapType(INTEGER, INTEGER), oneToNullMap); - Map sequenceToNullMap = new HashMap<>(); - sequenceToNullMap.put(1, null); - sequenceToNullMap.put(2, null); - sequenceToNullMap.put(3, null); - assertFunction("map_filter(map(ARRAY[1, 2, 3], ARRAY [NULL, NULL, NULL]), (k, v) -> v IS NULL)", new MapType(INTEGER, UNKNOWN), sequenceToNullMap); - assertFunction("map_filter(map(ARRAY[1, 2, 3], ARRAY [NULL, NULL, NULL]), (k, v) -> v IS NOT NULL)", new MapType(INTEGER, UNKNOWN), ImmutableMap.of()); - } - - @Test - public void testBasic() - throws Exception - { - assertFunction( - "map_filter(map(ARRAY [5, 6, 7, 8], ARRAY [5, 6, 6, 5]), (x, y) -> x <= 6 OR y = 5)", - new MapType(INTEGER, INTEGER), - ImmutableMap.of(5, 5, 6, 6, 8, 5)); - assertFunction( - "map_filter(map(ARRAY [5 + RANDOM(1), 6, 7, 8], ARRAY [5, 6, 6, 5]), (x, y) -> x <= 6 OR y = 5)", - new MapType(INTEGER, INTEGER), - ImmutableMap.of(5, 5, 6, 6, 8, 5)); - assertFunction( - "map_filter(map(ARRAY ['a', 'b', 'c', 'd'], ARRAY [1, 2, NULL, 4]), (k, v) -> v IS NOT NULL)", - new MapType(createVarcharType(1), INTEGER), - ImmutableMap.of("a", 1, "b", 2, "d", 4)); - assertFunction( - "map_filter(map(ARRAY ['a', 'b', 'c'], ARRAY [TRUE, FALSE, NULL]), (k, v) -> v)", - new MapType(createVarcharType(1), BOOLEAN), - ImmutableMap.of("a", true)); - } - - @Test - public void testTypeCombinations() - throws Exception - { - assertFunction( - "map_filter(map(ARRAY [25, 26, 27], ARRAY [25, 26, 27]), (k, v) -> k = 25 OR v = 27)", - new MapType(INTEGER, INTEGER), - ImmutableMap.of(25, 25, 27, 27)); - assertFunction( - "map_filter(map(ARRAY [25, 26, 27], ARRAY [25.5, 26.5, 27.5]), (k, v) -> k = 25 OR v = 27.5)", - new MapType(INTEGER, DOUBLE), - ImmutableMap.of(25, 25.5, 27, 27.5)); - assertFunction( - "map_filter(map(ARRAY [25, 26, 27], ARRAY [false, null, true]), (k, v) -> k = 25 OR v)", - new MapType(INTEGER, BOOLEAN), - ImmutableMap.of(25, false, 27, true)); - assertFunction( - "map_filter(map(ARRAY [25, 26, 27], ARRAY ['abc', 'def', 'xyz']), (k, v) -> k = 25 OR v = 'xyz')", - new MapType(INTEGER, createVarcharType(3)), - ImmutableMap.of(25, "abc", 27, "xyz")); - assertFunction( - "map_filter(map(ARRAY [25, 26, 27], ARRAY [ARRAY ['a', 'b'], ARRAY ['a', 'c'], ARRAY ['a', 'b', 'c']]), (k, v) -> k = 25 OR cardinality(v) = 3)", - new MapType(INTEGER, new ArrayType(createVarcharType(1))), - ImmutableMap.of(25, ImmutableList.of("a", "b"), 27, ImmutableList.of("a", "b", "c"))); - - assertFunction( - "map_filter(map(ARRAY [25.5, 26.5, 27.5], ARRAY [25, 26, 27]), (k, v) -> k = 25.5 OR v = 27)", - new MapType(DOUBLE, INTEGER), - ImmutableMap.of(25.5, 25, 27.5, 27)); - assertFunction( - "map_filter(map(ARRAY [25.5, 26.5, 27.5], ARRAY [25.5, 26.5, 27.5]), (k, v) -> k = 25.5 OR v = 27.5)", - new MapType(DOUBLE, DOUBLE), - ImmutableMap.of(25.5, 25.5, 27.5, 27.5)); - assertFunction( - "map_filter(map(ARRAY [25.5, 26.5, 27.5], ARRAY [false, null, true]), (k, v) -> k = 25.5 OR v)", - new MapType(DOUBLE, BOOLEAN), - ImmutableMap.of(25.5, false, 27.5, true)); - assertFunction( - "map_filter(map(ARRAY [25.5, 26.5, 27.5], ARRAY ['abc', 'def', 'xyz']), (k, v) -> k = 25.5 OR v = 'xyz')", - new MapType(DOUBLE, createVarcharType(3)), - ImmutableMap.of(25.5, "abc", 27.5, "xyz")); - assertFunction( - "map_filter(map(ARRAY [25.5, 26.5, 27.5], ARRAY [ARRAY ['a', 'b'], ARRAY ['a', 'c'], ARRAY ['a', 'b', 'c']]), (k, v) -> k = 25.5 OR cardinality(v) = 3)", - new MapType(DOUBLE, new ArrayType(createVarcharType(1))), - ImmutableMap.of(25.5, ImmutableList.of("a", "b"), 27.5, ImmutableList.of("a", "b", "c"))); - - assertFunction( - "map_filter(map(ARRAY [true, false], ARRAY [25, 26]), (k, v) -> k AND v = 25)", - new MapType(BOOLEAN, INTEGER), - ImmutableMap.of(true, 25)); - assertFunction( - "map_filter(map(ARRAY [false, true], ARRAY [25.5, 26.5]), (k, v) -> k OR v > 100)", - new MapType(BOOLEAN, DOUBLE), - ImmutableMap.of(true, 26.5)); - Map falseToNullMap = new HashMap<>(); - falseToNullMap.put(false, null); - assertFunction( - "map_filter(map(ARRAY [true, false], ARRAY [false, null]), (k, v) -> NOT k OR v)", - new MapType(BOOLEAN, BOOLEAN), - falseToNullMap); - assertFunction( - "map_filter(map(ARRAY [false, true], ARRAY ['abc', 'def']), (k, v) -> NOT k AND v = 'abc')", - new MapType(BOOLEAN, createVarcharType(3)), - ImmutableMap.of(false, "abc")); - assertFunction( - "map_filter(map(ARRAY [true, false], ARRAY [ARRAY ['a', 'b'], ARRAY ['a', 'b', 'c']]), (k, v) -> k OR cardinality(v) = 3)", - new MapType(BOOLEAN, new ArrayType(createVarcharType(1))), - ImmutableMap.of(true, ImmutableList.of("a", "b"), false, ImmutableList.of("a", "b", "c"))); - - assertFunction( - "map_filter(map(ARRAY ['s0', 's1', 's2'], ARRAY [25, 26, 27]), (k, v) -> k = 's0' OR v = 27)", - new MapType(createVarcharType(2), INTEGER), - ImmutableMap.of("s0", 25, "s2", 27)); - assertFunction( - "map_filter(map(ARRAY ['s0', 's1', 's2'], ARRAY [25.5, 26.5, 27.5]), (k, v) -> k = 's0' OR v = 27.5)", - new MapType(createVarcharType(2), DOUBLE), - ImmutableMap.of("s0", 25.5, "s2", 27.5)); - assertFunction( - "map_filter(map(ARRAY ['s0', 's1', 's2'], ARRAY [false, null, true]), (k, v) -> k = 's0' OR v)", - new MapType(createVarcharType(2), BOOLEAN), - ImmutableMap.of("s0", false, "s2", true)); - assertFunction( - "map_filter(map(ARRAY ['s0', 's1', 's2'], ARRAY ['abc', 'def', 'xyz']), (k, v) -> k = 's0' OR v = 'xyz')", - new MapType(createVarcharType(2), createVarcharType(3)), - ImmutableMap.of("s0", "abc", "s2", "xyz")); - assertFunction( - "map_filter(map(ARRAY ['s0', 's1', 's2'], ARRAY [ARRAY ['a', 'b'], ARRAY ['a', 'c'], ARRAY ['a', 'b', 'c']]), (k, v) -> k = 's0' OR cardinality(v) = 3)", - new MapType(createVarcharType(2), new ArrayType(createVarcharType(1))), - ImmutableMap.of("s0", ImmutableList.of("a", "b"), "s2", ImmutableList.of("a", "b", "c"))); - - assertFunction( - "map_filter(map(ARRAY [ARRAY [1, 2], ARRAY [3, 4], ARRAY []], ARRAY [25, 26, 27]), (k, v) -> k = ARRAY [1, 2] OR v = 27)", - new MapType(new ArrayType(INTEGER), INTEGER), - ImmutableMap.of(ImmutableList.of(1, 2), 25, ImmutableList.of(), 27)); - assertFunction( - "map_filter(map(ARRAY [ARRAY [1, 2], ARRAY [3, 4], ARRAY []], ARRAY [25.5, 26.5, 27.5]), (k, v) -> k = ARRAY [1, 2] OR v = 27.5)", - new MapType(new ArrayType(INTEGER), DOUBLE), - ImmutableMap.of(ImmutableList.of(1, 2), 25.5, ImmutableList.of(), 27.5)); - assertFunction( - "map_filter(map(ARRAY [ARRAY [1, 2], ARRAY [3, 4], ARRAY []], ARRAY [false, null, true]), (k, v) -> k = ARRAY [1, 2] OR v)", - new MapType(new ArrayType(INTEGER), BOOLEAN), - ImmutableMap.of(ImmutableList.of(1, 2), false, ImmutableList.of(), true)); - assertFunction( - "map_filter(map(ARRAY [ARRAY [1, 2], ARRAY [3, 4], ARRAY []], ARRAY ['abc', 'def', 'xyz']), (k, v) -> k = ARRAY [1, 2] OR v = 'xyz')", - new MapType(new ArrayType(INTEGER), createVarcharType(3)), - ImmutableMap.of(ImmutableList.of(1, 2), "abc", ImmutableList.of(), "xyz")); - assertFunction( - "map_filter(map(ARRAY [ARRAY [1, 2], ARRAY [3, 4], ARRAY []], ARRAY [ARRAY ['a', 'b'], ARRAY ['a', 'b', 'c'], ARRAY ['a', 'c']]), (k, v) -> cardinality(k) = 0 OR cardinality(v) = 3)", - new MapType(new ArrayType(INTEGER), new ArrayType(createVarcharType(1))), - ImmutableMap.of(ImmutableList.of(3, 4), ImmutableList.of("a", "b", "c"), ImmutableList.of(), ImmutableList.of("a", "c"))); - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestStringFunctions.java b/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestStringFunctions.java index 611c79abfaef..fe0b7a3f45a8 100644 --- a/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestStringFunctions.java +++ b/presto-main/src/test/java/com/facebook/presto/operator/scalar/TestStringFunctions.java @@ -420,7 +420,7 @@ public void testLeftTrimParametrized() assertInvalidFunction("LTRIM('hello wolrd', utf8(from_hex('3281')))", "Invalid UTF-8 encoding in characters to trim: 2�"); } - private static SqlVarbinary varbinary(int... bytesAsInts) + private SqlVarbinary varbinary(int... bytesAsInts) { byte[] bytes = new byte[bytesAsInts.length]; for (int i = 0; i < bytes.length; i++) { diff --git a/presto-main/src/test/java/com/facebook/presto/security/TestAccessControlManager.java b/presto-main/src/test/java/com/facebook/presto/security/TestAccessControlManager.java index 01945e0622ac..def7dc6ab4fa 100644 --- a/presto-main/src/test/java/com/facebook/presto/security/TestAccessControlManager.java +++ b/presto-main/src/test/java/com/facebook/presto/security/TestAccessControlManager.java @@ -68,7 +68,7 @@ public void testNoneSystemAccessControl() throws Exception { AccessControlManager accessControlManager = new AccessControlManager(createTestTransactionManager()); - accessControlManager.setSystemAccessControl(ALLOW_ALL_ACCESS_CONTROL, ImmutableMap.of()); + accessControlManager.setSystemAccessControl(ALLOW_ALL_ACCESS_CONTROL, ImmutableMap.of()); accessControlManager.checkCanSetUser(null, USER_NAME); } diff --git a/presto-main/src/test/java/com/facebook/presto/server/TestExecuteResource.java b/presto-main/src/test/java/com/facebook/presto/server/TestExecuteResource.java new file mode 100644 index 000000000000..e756274295c3 --- /dev/null +++ b/presto-main/src/test/java/com/facebook/presto/server/TestExecuteResource.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.server; + +import com.facebook.presto.client.PrestoHeaders; +import com.facebook.presto.server.testing.TestingPrestoServer; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.net.HttpHeaders; +import io.airlift.http.client.HttpClient; +import io.airlift.http.client.HttpStatus; +import io.airlift.http.client.Request; +import io.airlift.http.client.jetty.JettyHttpClient; +import io.airlift.testing.Closeables; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import static io.airlift.http.client.Request.Builder.preparePost; +import static io.airlift.http.client.StaticBodyGenerator.createStaticBodyGenerator; +import static io.airlift.http.client.StringResponseHandler.StringResponse; +import static io.airlift.http.client.StringResponseHandler.createStringResponseHandler; +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.testng.Assert.assertEquals; + +@Test(singleThreaded = true) +public class TestExecuteResource +{ + private TestingPrestoServer server; + private HttpClient client; + + @BeforeMethod + public void setup() + throws Exception + { + server = new TestingPrestoServer(); + client = new JettyHttpClient(); + } + + @SuppressWarnings("deprecation") + @AfterMethod + public void teardown() + { + Closeables.closeQuietly(server); + Closeables.closeQuietly(client); + } + + @Test + public void testExecute() + throws Exception + { + String expected = "{\"columns\":[" + + "{\"name\":\"foo\",\"type\":\"integer\",\"typeSignature\":{\"rawType\":\"integer\",\"arguments\":[],\"typeArguments\":[],\"literalArguments\":[]}}," + + "{\"name\":\"phoo\",\"type\":\"bigint\",\"typeSignature\":{\"rawType\":\"bigint\",\"arguments\":[],\"typeArguments\":[],\"literalArguments\":[]}}," + + "{\"name\":\"bar\",\"type\":\"varchar(3)\",\"typeSignature\":" + + "{\"rawType\":\"varchar\",\"arguments\":[{\"kind\":\"LONG_LITERAL\",\"value\":3}],\"typeArguments\":[],\"literalArguments\":[]}}," + + "{\"name\":\"baz\",\"type\":\"array(bigint)\",\"typeSignature\":" + + "{\"rawType\":\"array\",\"arguments\":[{\"kind\":\"TYPE_SIGNATURE\",\"value\":{\"rawType\":\"bigint\",\"arguments\":[],\"typeArguments\":[],\"literalArguments\":[]}}],\"typeArguments\":[{\"rawType\":\"bigint\",\"arguments\":[],\"typeArguments\":[],\"literalArguments\":[]}],\"literalArguments\":[]}}]," + + "\"data\":[[123, 12300000000, \"abc\",[42,44]]]}\n"; + StringResponse response = executeQuery("SELECT 123 foo, 12300000000 phoo, 'abc' bar, CAST(JSON_PARSE('[42,44]') AS ARRAY) baz"); + assertEquals(response.getStatusCode(), HttpStatus.OK.code()); + assertEquals(response.getHeader(HttpHeaders.CONTENT_TYPE), "application/json"); + + ObjectMapper mapper = new ObjectMapper(); + assertEquals(mapper.readTree(response.getBody()), mapper.readTree(expected)); + } + + private StringResponse executeQuery(String query) + { + Request request = preparePost() + .setUri(server.resolve("/v1/execute")) + .setHeader(PrestoHeaders.PRESTO_USER, "test") + .setHeader(PrestoHeaders.PRESTO_CATALOG, "catalog") + .setHeader(PrestoHeaders.PRESTO_SCHEMA, "schema") + .setHeader(PrestoHeaders.PRESTO_TIME_ZONE, "UTC") + .setBodyGenerator(createStaticBodyGenerator(query, UTF_8)) + .build(); + return client.execute(request, createStringResponseHandler()); + } +} diff --git a/presto-main/src/test/java/com/facebook/presto/server/remotetask/TestHttpRemoteTask.java b/presto-main/src/test/java/com/facebook/presto/server/remotetask/TestHttpRemoteTask.java deleted file mode 100644 index 4ac2ee92c270..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/server/remotetask/TestHttpRemoteTask.java +++ /dev/null @@ -1,232 +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.server.remotetask; - -import com.facebook.presto.OutputBuffers; -import com.facebook.presto.client.NodeVersion; -import com.facebook.presto.execution.NodeTaskMap; -import com.facebook.presto.execution.QueryManagerConfig; -import com.facebook.presto.execution.RemoteTask; -import com.facebook.presto.execution.TaskId; -import com.facebook.presto.execution.TaskInfo; -import com.facebook.presto.execution.TaskManagerConfig; -import com.facebook.presto.execution.TaskState; -import com.facebook.presto.execution.TaskStatus; -import com.facebook.presto.execution.TaskTestUtils; -import com.facebook.presto.execution.TestSqlTaskManager; -import com.facebook.presto.metadata.PrestoNode; -import com.facebook.presto.server.HttpRemoteTaskFactory; -import com.facebook.presto.server.TaskUpdateRequest; -import com.google.common.collect.ImmutableListMultimap; -import com.google.common.collect.ImmutableMultimap; -import io.airlift.http.client.HttpStatus; -import io.airlift.http.client.Request; -import io.airlift.http.client.Response; -import io.airlift.http.client.testing.TestingHttpClient; -import io.airlift.http.client.testing.TestingResponse; -import io.airlift.json.JsonCodec; -import io.airlift.units.Duration; -import org.testng.annotations.Test; - -import java.net.URI; -import java.nio.charset.StandardCharsets; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.atomic.AtomicLong; -import java.util.function.BiConsumer; - -import static com.facebook.presto.OutputBuffers.createInitialEmptyOutputBuffers; -import static com.facebook.presto.SessionTestUtils.TEST_SESSION; -import static com.facebook.presto.client.PrestoHeaders.PRESTO_MAX_WAIT; -import static com.facebook.presto.client.PrestoHeaders.PRESTO_TASK_INSTANCE_ID; -import static com.facebook.presto.spi.StandardErrorCode.REMOTE_TASK_MISMATCH; -import static com.google.common.collect.Iterables.getOnlyElement; -import static java.lang.String.format; -import static java.util.concurrent.TimeUnit.MILLISECONDS; -import static java.util.concurrent.TimeUnit.SECONDS; -import static javax.ws.rs.core.HttpHeaders.CONTENT_TYPE; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertTrue; - -public class TestHttpRemoteTask -{ - // This timeout should never be reached because a daemon thread in test should fail the test and do proper cleanup. - @Test(timeOut = 30000) - public void testRemoteTaskMismatch() - throws InterruptedException, ExecutionException - { - Duration idleTimeout = new Duration(3, SECONDS); - Duration failTimeout = new Duration(20, SECONDS); - - JsonCodec taskStatusCodec = JsonCodec.jsonCodec(TaskStatus.class); - JsonCodec taskInfoCodec = JsonCodec.jsonCodec(TaskInfo.class); - TaskManagerConfig taskManagerConfig = new TaskManagerConfig(); - - // Shorten status refresh wait and info update interval so that we can have a shorter test timeout - taskManagerConfig.setStatusRefreshMaxWait(new Duration(idleTimeout.roundTo(MILLISECONDS) / 100, MILLISECONDS)); - taskManagerConfig.setInfoUpdateInterval(new Duration(idleTimeout.roundTo(MILLISECONDS) / 10, MILLISECONDS)); - - AtomicLong lastActivityNanos = new AtomicLong(System.nanoTime()); - HttpProcessor httpProcessor = new HttpProcessor(taskStatusCodec, taskInfoCodec, lastActivityNanos); - TestingHttpClient testingHttpClient = new TestingHttpClient(httpProcessor); - - HttpRemoteTaskFactory httpRemoteTaskFactory = new HttpRemoteTaskFactory( - new QueryManagerConfig(), - taskManagerConfig, - testingHttpClient, - new TestSqlTaskManager.MockLocationFactory(), - taskStatusCodec, - taskInfoCodec, - JsonCodec.jsonCodec(TaskUpdateRequest.class), - new RemoteTaskStats()); - RemoteTask remoteTask = httpRemoteTaskFactory.createRemoteTask( - TEST_SESSION, - new TaskId("test", 1, 2), - new PrestoNode("node-id", URI.create("http://192.0.1.2"), new NodeVersion("version"), false), - TaskTestUtils.PLAN_FRAGMENT, - ImmutableMultimap.of(), - createInitialEmptyOutputBuffers(OutputBuffers.BufferType.BROADCAST), - new NodeTaskMap.PartitionedSplitCountTracker(i -> { }), - true); - - httpProcessor.setInitialTaskInfo(remoteTask.getTaskInfo()); - remoteTask.start(); - - CompletableFuture testComplete = new CompletableFuture<>(); - asyncRun( - idleTimeout.roundTo(MILLISECONDS), - failTimeout.roundTo(MILLISECONDS), - lastActivityNanos, - () -> testComplete.complete(null), - (message, cause) -> testComplete.completeExceptionally(new AssertionError(message, cause))); - testComplete.get(); - - httpRemoteTaskFactory.stop(); - assertTrue(remoteTask.getTaskStatus().getState().isDone(), format("TaskStatus is not in a done state: %s", remoteTask.getTaskStatus())); - assertEquals(getOnlyElement(remoteTask.getTaskStatus().getFailures()).getErrorCode(), REMOTE_TASK_MISMATCH.toErrorCode()); - assertTrue(remoteTask.getTaskInfo().getTaskStatus().getState().isDone(), format("TaskInfo is not in a done state: %s", remoteTask.getTaskInfo())); - } - - private static void asyncRun(long idleTimeoutMillis, long failTimeoutMillis, AtomicLong lastActivityNanos, Runnable runAfterIdle, BiConsumer runAfterFail) - { - new Thread(() -> { - long startTimeNanos = System.nanoTime(); - - try { - while (true) { - long millisSinceLastActivity = (System.nanoTime() - lastActivityNanos.get()) / 1_000_000L; - long millisSinceStart = (System.nanoTime() - startTimeNanos) / 1_000_000L; - long millisToIdleTarget = idleTimeoutMillis - millisSinceLastActivity; - long millisToFailTarget = failTimeoutMillis - millisSinceStart; - if (millisToFailTarget < millisToIdleTarget) { - runAfterFail.accept(format("Activity doesn't stop after %sms", failTimeoutMillis), null); - return; - } - if (millisToIdleTarget < 0) { - runAfterIdle.run(); - return; - } - Thread.sleep(millisToIdleTarget); - } - } - catch (InterruptedException e) { - runAfterFail.accept("Idle/fail timeout monitor thread interrupted", e); - } - }).start(); - } - - private static class HttpProcessor implements TestingHttpClient.Processor - { - private static final String INITIAL_TASK_INSTANCE_ID = "task-instance-id"; - private static final String NEW_TASK_INSTANCE_ID = "task-instance-id-x"; - private final JsonCodec taskStatusCodec; - private final JsonCodec taskInfoCodec; - private final AtomicLong lastActivityNanos; - - private TaskInfo initialTaskInfo; - private TaskStatus initialTaskStatus; - private long version; - private TaskState taskState; - - private long statusFetchCounter; - private String taskInstanceId = INITIAL_TASK_INSTANCE_ID; - - public HttpProcessor(JsonCodec taskStatusCodec, JsonCodec taskInfoCodec, AtomicLong lastActivityNanos) - { - this.taskStatusCodec = taskStatusCodec; - this.taskInfoCodec = taskInfoCodec; - this.lastActivityNanos = lastActivityNanos; - } - - @Override - public synchronized Response handle(Request request) - throws Exception - { - lastActivityNanos.set(System.nanoTime()); - - ImmutableListMultimap.Builder headers = ImmutableListMultimap.builder(); - headers.put(PRESTO_TASK_INSTANCE_ID, taskInstanceId); - headers.put(CONTENT_TYPE, "application/json"); - - if (request.getUri().getPath().endsWith("/status")) { - statusFetchCounter++; - if (statusFetchCounter >= 10) { - // Change the task instance id after 10th fetch to simulate worker restart - taskInstanceId = NEW_TASK_INSTANCE_ID; - } - wait(Duration.valueOf(request.getHeader(PRESTO_MAX_WAIT)).roundTo(MILLISECONDS)); - return new TestingResponse(HttpStatus.OK, headers.build(), taskStatusCodec.toJson(buildTaskStatus()).getBytes(StandardCharsets.UTF_8)); - } - if ("DELETE".equals(request.getMethod())) { - taskState = TaskState.ABORTED; - } - return new TestingResponse(HttpStatus.OK, headers.build(), taskInfoCodec.toJson(buildTaskInfo()).getBytes(StandardCharsets.UTF_8)); - } - - public void setInitialTaskInfo(TaskInfo initialTaskInfo) - { - this.initialTaskInfo = initialTaskInfo; - this.initialTaskStatus = initialTaskInfo.getTaskStatus(); - this.taskState = initialTaskStatus.getState(); - this.version = initialTaskStatus.getVersion(); - } - - private TaskInfo buildTaskInfo() - { - return new TaskInfo( - buildTaskStatus(), - initialTaskInfo.getLastHeartbeat(), - initialTaskInfo.getOutputBuffers(), - initialTaskInfo.getNoMoreSplits(), - initialTaskInfo.getStats(), - initialTaskInfo.isNeedsPlan(), - initialTaskInfo.isComplete()); - } - - private TaskStatus buildTaskStatus() - { - return new TaskStatus( - initialTaskStatus.getTaskId(), - taskInstanceId, - ++version, - taskState, - initialTaskStatus.getSelf(), - initialTaskStatus.getFailures(), - initialTaskStatus.getQueuedPartitionedDrivers(), - initialTaskStatus.getRunningPartitionedDrivers(), - initialTaskStatus.getMemoryReservation()); - } - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/spiller/TestBinaryFileSpiller.java b/presto-main/src/test/java/com/facebook/presto/spiller/TestBinaryFileSpiller.java index 4968ae31b4be..ffe661269fd4 100644 --- a/presto-main/src/test/java/com/facebook/presto/spiller/TestBinaryFileSpiller.java +++ b/presto-main/src/test/java/com/facebook/presto/spiller/TestBinaryFileSpiller.java @@ -38,7 +38,6 @@ import static java.lang.Double.doubleToLongBits; import static org.testng.Assert.assertEquals; -@Test(singleThreaded = true) public class TestBinaryFileSpiller { private static final List TYPES = ImmutableList.of(BIGINT, VARCHAR, DOUBLE, BIGINT); @@ -96,15 +95,12 @@ private void testSimpleSpiller(Spiller spiller) private void testSpiller(List types, Spiller spiller, List... spills) throws ExecutionException, InterruptedException { - long spilledBytesBefore = factory.getSpilledBytes(); - long spilledBytes = 0; for (List spill : spills) { - spilledBytes += spill.stream().mapToLong(Page::getSizeInBytes).sum(); spiller.spill(spill.iterator()).get(); } - assertEquals(factory.getSpilledBytes() - spilledBytesBefore, spilledBytes); List> actualSpills = spiller.getSpills(); + assertEquals(actualSpills.size(), spills.length); for (int i = 0; i < actualSpills.size(); i++) { diff --git a/presto-main/src/test/java/com/facebook/presto/sql/TestExpressionInterpreter.java b/presto-main/src/test/java/com/facebook/presto/sql/TestExpressionInterpreter.java index 967260b8764e..99f10193a04b 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/TestExpressionInterpreter.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/TestExpressionInterpreter.java @@ -1073,7 +1073,7 @@ public void testLike() public void testLikeOptimization() throws Exception { - assertOptimizedEquals("unbound_string like 'abc'", "unbound_string = CAST('abc' AS VARCHAR)"); + assertOptimizedEquals("unbound_string like 'abc'", "unbound_string = 'abc'"); assertOptimizedEquals("bound_string like bound_pattern", "true"); assertOptimizedEquals("'abc' like bound_pattern", "false"); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestAnalyzer.java b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestAnalyzer.java index b0cb26ff8ed5..b4466613d867 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestAnalyzer.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestAnalyzer.java @@ -59,7 +59,6 @@ import static com.facebook.presto.connector.ConnectorId.createInformationSchemaConnectorId; import static com.facebook.presto.connector.ConnectorId.createSystemTablesConnectorId; import static com.facebook.presto.metadata.ViewDefinition.ViewColumn; -import static com.facebook.presto.operator.scalar.ApplyFunction.APPLY_FUNCTION; import static com.facebook.presto.spi.type.BigintType.BIGINT; import static com.facebook.presto.spi.type.DoubleType.DOUBLE; import static com.facebook.presto.spi.type.VarcharType.VARCHAR; @@ -90,7 +89,6 @@ import static com.facebook.presto.sql.analyzer.SemanticErrorCode.ORDER_BY_MUST_BE_IN_SELECT; import static com.facebook.presto.sql.analyzer.SemanticErrorCode.SAMPLE_PERCENTAGE_OUT_OF_RANGE; import static com.facebook.presto.sql.analyzer.SemanticErrorCode.SCHEMA_NOT_SPECIFIED; -import static com.facebook.presto.sql.analyzer.SemanticErrorCode.STANDALONE_LAMBDA; import static com.facebook.presto.sql.analyzer.SemanticErrorCode.TYPE_MISMATCH; import static com.facebook.presto.sql.analyzer.SemanticErrorCode.VIEW_IS_STALE; import static com.facebook.presto.sql.analyzer.SemanticErrorCode.WILDCARD_WITHOUT_FROM; @@ -337,8 +335,7 @@ public void testMultipleDistinctAggregations() public void testOrderByExpressionOnOutputColumn() throws Exception { - // TODO: analyze output - analyze("SELECT a x FROM t1 ORDER BY x + 1"); + assertFails(MISSING_ATTRIBUTE, "SELECT a x FROM t1 ORDER BY x + 1"); } @Test @@ -354,7 +351,7 @@ public void testOrderByWithWildcard() throws Exception { // TODO: validate output - analyze("SELECT t1.* FROM t1 ORDER BY a"); + analyze("SELECT a, t1.* FROM t1 ORDER BY a"); } @Test @@ -406,8 +403,6 @@ public void testAmbiguousReferenceInOrderBy() throws Exception { assertFails(AMBIGUOUS_ATTRIBUTE, "SELECT a x, b x FROM t1 ORDER BY x"); - assertFails(AMBIGUOUS_ATTRIBUTE, "SELECT a x, a x FROM t1 ORDER BY x"); - assertFails(AMBIGUOUS_ATTRIBUTE, "SELECT a, a FROM t1 ORDER BY a"); } @Test @@ -953,8 +948,7 @@ public void testLiteral() public void testLambda() throws Exception { - analyze("SELECT apply(5, x -> abs(x)) from t1"); - assertFails(STANDALONE_LAMBDA, "SELECT x -> abs(x) from t1"); + assertFails(NOT_SUPPORTED, "SELECT x -> abs(x) from t1"); } @Test @@ -1058,8 +1052,6 @@ public void setup() new TablePropertyManager(), transactionManager); - metadata.getFunctionRegistry().addFunctions(ImmutableList.of(APPLY_FUNCTION)); - catalogManager.registerCatalog(createTestingCatalog(TPCH_CATALOG, TPCH_CONNECTOR_ID)); catalogManager.registerCatalog(createTestingCatalog(SECOND_CATALOG, SECOND_CONNECTOR_ID)); catalogManager.registerCatalog(createTestingCatalog(THIRD_CATALOG, THIRD_CONNECTOR_ID)); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java index 16e54d9117f1..032ecb9d6be0 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestFeaturesConfig.java @@ -54,8 +54,7 @@ public void testDefaults() .setOperatorMemoryLimitBeforeSpill(DataSize.valueOf("4MB")) .setSpillerSpillPath(Paths.get(System.getProperty("java.io.tmpdir"), "presto", "spills").toString()) .setSpillerThreads(4) - .setOptimizeMixedDistinctAggregations(false) - .setLegacyOrderBy(false)); + .setOptimizeMixedDistinctAggregations(false)); } @Test @@ -64,7 +63,6 @@ public void testExplicitPropertyMappings() Map propertiesLegacy = new ImmutableMap.Builder() .put("experimental.resource-groups-enabled", "true") .put("deprecated.legacy-array-agg", "true") - .put("deprecated.legacy-order-by", "true") .put("distributed-index-joins-enabled", "true") .put("distributed-joins-enabled", "false") .put("colocated-joins-enabled", "true") @@ -87,7 +85,6 @@ public void testExplicitPropertyMappings() Map properties = new ImmutableMap.Builder() .put("experimental.resource-groups-enabled", "true") .put("deprecated.legacy-array-agg", "true") - .put("deprecated.legacy-order-by", "true") .put("distributed-index-joins-enabled", "true") .put("distributed-joins-enabled", "false") .put("colocated-joins-enabled", "true") @@ -128,8 +125,7 @@ public void testExplicitPropertyMappings() .setSpillEnabled(true) .setOperatorMemoryLimitBeforeSpill(DataSize.valueOf("100MB")) .setSpillerSpillPath("/tmp/custom/spill/path") - .setSpillerThreads(42) - .setLegacyOrderBy(true); + .setSpillerThreads(42); assertFullMapping(properties, expected); assertDeprecatedEquivalence(FeaturesConfig.class, properties, propertiesLegacy); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestScope.java b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestScope.java index 4691930c933d..53cdf0d7c2a0 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestScope.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/analyzer/TestScope.java @@ -81,7 +81,7 @@ public void test() assertEquals(inner2.tryResolveField(c4).get().isLocal(), true); } - private static QualifiedNameReference name(String first, String... parts) + private QualifiedNameReference name(String first, String... parts) { return new QualifiedNameReference(QualifiedName.of(first, parts)); } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/gen/PageProcessorBenchmark.java b/presto-main/src/test/java/com/facebook/presto/sql/gen/PageProcessorBenchmark.java index af2f8e1e2a14..737c455feb76 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/gen/PageProcessorBenchmark.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/gen/PageProcessorBenchmark.java @@ -26,6 +26,7 @@ import com.facebook.presto.sql.relational.RowExpression; import com.facebook.presto.sql.relational.SqlToRowExpressionTranslator; import com.facebook.presto.sql.tree.Expression; +import com.facebook.presto.sql.tree.ExpressionTreeRewriter; import com.facebook.presto.testing.TestingSession; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -143,7 +144,7 @@ private RowExpression getFilter(Type type) private List getProjections(Type type) { - ImmutableList.Builder builder = ImmutableList.builder(); + ImmutableList.Builder builder = ImmutableList.builder(); if (type == BIGINT) { for (int i = 0; i < columnCount; i++) { builder.add(rowExpression("bigint" + i + " + 5", type)); @@ -162,7 +163,7 @@ else if (type == VARCHAR) { private RowExpression rowExpression(String expression, Type type) { SymbolToInputRewriter symbolToInputRewriter = new SymbolToInputRewriter(sourceLayout); - Expression inputReferenceExpression = symbolToInputRewriter.rewrite(createExpression(expression, METADATA, symbolTypes)); + Expression inputReferenceExpression = ExpressionTreeRewriter.rewriteWith(symbolToInputRewriter, createExpression(expression, METADATA, symbolTypes)); ImmutableMap.Builder builder = ImmutableMap.builder(); for (int i = 0; i < columnCount; i++) { diff --git a/presto-main/src/test/java/com/facebook/presto/sql/gen/TestExpressionCompiler.java b/presto-main/src/test/java/com/facebook/presto/sql/gen/TestExpressionCompiler.java index 4db9335ba2aa..09aa4b94693f 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/gen/TestExpressionCompiler.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/gen/TestExpressionCompiler.java @@ -954,7 +954,7 @@ else if (secondTest != null && (double) value == secondTest) { expected = "else"; } List expressions = formatExpression("case when %s = %s then 'first' when %s = %s then 'second' else 'else' end", - Arrays.asList(value, firstTest, value, secondTest), + Arrays.asList(value, firstTest, value, secondTest), ImmutableList.of("double", "bigint", "double", "double")); assertExecute(expressions, createVarcharType(6), expected); } @@ -985,7 +985,7 @@ else if (secondTest != null && (double) value == secondTest) { expected = null; } List expressions = formatExpression("case when %s = %s then 'first' when %s = %s then 'second' end", - Arrays.asList(value, firstTest, value, secondTest), + Arrays.asList(value, firstTest, value, secondTest), ImmutableList.of("double", "bigint", "double", "double")); assertExecute(expressions, createVarcharType(6), expected); } @@ -1535,31 +1535,31 @@ private List generateExpression(String expressionPattern, String first, return formatExpression(expressionPattern, first, "varchar", second, "integer", third, "integer"); } - private static List formatExpression(String expressionPattern, Object value, String type) + private List formatExpression(String expressionPattern, Object value, String type) { return formatExpression(expressionPattern, - Arrays.asList(value), + Arrays.asList(value), ImmutableList.of(type)); } - private static List formatExpression(String expressionPattern, Object left, final String leftType, Object right, final String rightType) + private List formatExpression(String expressionPattern, Object left, final String leftType, Object right, final String rightType) { return formatExpression(expressionPattern, - Arrays.asList(left, right), + Arrays.asList(left, right), ImmutableList.of(leftType, rightType)); } - private static List formatExpression(String expressionPattern, + private List formatExpression(String expressionPattern, Object first, String firstType, Object second, String secondType, Object third, String thirdType) { return formatExpression(expressionPattern, - Arrays.asList(first, second, third), + Arrays.asList(first, second, third), ImmutableList.of(firstType, secondType, thirdType)); } - private static List formatExpression(String expressionPattern, List values, List types) + private List formatExpression(String expressionPattern, List values, List types) { Preconditions.checkArgument(values.size() == types.size()); @@ -1633,7 +1633,7 @@ private void assertExecute(List expressions, BigDecimal decimal) } } - private static Type getDecimalType(BigDecimal decimal) + private Type getDecimalType(BigDecimal decimal) { if (decimal == null) { return createDecimalType(1, 0); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/gen/TestJoinCompiler.java b/presto-main/src/test/java/com/facebook/presto/sql/gen/TestJoinCompiler.java index c7e05bb64d3f..a0a0edecaf2f 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/gen/TestJoinCompiler.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/gen/TestJoinCompiler.java @@ -55,7 +55,7 @@ public static Object[][] hashEnabledValuesProvider() public void testSingleChannel(boolean hashEnabled) throws Exception { - List joinTypes = ImmutableList.of(VARCHAR); + List joinTypes = ImmutableList.of(VARCHAR); List joinChannels = Ints.asList(0); // compile a single channel hash strategy @@ -137,8 +137,8 @@ public void testMultiChannel(boolean hashEnabled) { // compile a single channel hash strategy JoinCompiler joinCompiler = new JoinCompiler(); - List types = ImmutableList.of(VARCHAR, VARCHAR, BIGINT, DOUBLE, BOOLEAN); - List joinTypes = ImmutableList.of(VARCHAR, BIGINT, DOUBLE, BOOLEAN); + List types = ImmutableList.of(VARCHAR, VARCHAR, BIGINT, DOUBLE, BOOLEAN); + List joinTypes = ImmutableList.of(VARCHAR, BIGINT, DOUBLE, BOOLEAN); List joinChannels = Ints.asList(1, 2, 3, 4); // crate hash strategy with a single channel blocks -- make sure there is some overlap in values @@ -174,7 +174,7 @@ public void testMultiChannel(boolean hashEnabled) hashChannel = Optional.of(5); precomputedHash = hashChannelBuilder.build(); channels = ImmutableList.of(extraChannel, varcharChannel, longChannel, doubleChannel, booleanChannel, precomputedHash); - types = ImmutableList.of(VARCHAR, VARCHAR, BIGINT, DOUBLE, BOOLEAN, BIGINT); + types = ImmutableList.of(VARCHAR, VARCHAR, BIGINT, DOUBLE, BOOLEAN, BIGINT); } PagesHashStrategyFactory pagesHashStrategyFactory = joinCompiler.compilePagesHashStrategyFactory(types, joinChannels); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/gen/TestJoinProbeCompiler.java b/presto-main/src/test/java/com/facebook/presto/sql/gen/TestJoinProbeCompiler.java index 185ee326e051..a2609b42fd3b 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/gen/TestJoinProbeCompiler.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/gen/TestJoinProbeCompiler.java @@ -80,7 +80,7 @@ public void testSingleChannel(boolean hashEnabled) { taskContext.addPipelineContext(true, true).addDriverContext(); - ImmutableList types = ImmutableList.of(VARCHAR); + ImmutableList types = ImmutableList.of(VARCHAR); LookupSourceSupplierFactory lookupSourceSupplierFactory = joinCompiler.compileLookupSourceFactory(types, Ints.asList(0)); // crate hash strategy with a single channel blocks -- make sure there is some overlap in values @@ -104,7 +104,7 @@ public void testSingleChannel(boolean hashEnabled) for (Block block : channel) { hashChannelBuilder.add(TypeUtils.getHashBlock(ImmutableList.of(VARCHAR), block)); } - types = ImmutableList.of(VARCHAR, BigintType.BIGINT); + types = ImmutableList.of(VARCHAR, BigintType.BIGINT); hashChannel = Optional.of(1); channels = ImmutableList.of(channel, hashChannelBuilder.build()); } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestDomainTranslator.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestDomainTranslator.java index 5e2b9efab137..9301c29730f7 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestDomainTranslator.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestDomainTranslator.java @@ -19,7 +19,6 @@ 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.DecimalType; import com.facebook.presto.spi.type.StandardTypes; import com.facebook.presto.spi.type.Type; import com.facebook.presto.sql.planner.DomainTranslator.ExtractionResult; @@ -27,6 +26,7 @@ import com.facebook.presto.sql.tree.Cast; import com.facebook.presto.sql.tree.ComparisonExpression; import com.facebook.presto.sql.tree.ComparisonExpressionType; +import com.facebook.presto.sql.tree.DecimalLiteral; import com.facebook.presto.sql.tree.DoubleLiteral; import com.facebook.presto.sql.tree.Expression; import com.facebook.presto.sql.tree.FunctionCall; @@ -61,19 +61,14 @@ import static com.facebook.presto.spi.type.CharType.createCharType; import static com.facebook.presto.spi.type.DateType.DATE; import static com.facebook.presto.spi.type.DecimalType.createDecimalType; -import static com.facebook.presto.spi.type.Decimals.encodeScaledValue; import static com.facebook.presto.spi.type.DoubleType.DOUBLE; import static com.facebook.presto.spi.type.HyperLogLogType.HYPER_LOG_LOG; import static com.facebook.presto.spi.type.IntegerType.INTEGER; -import static com.facebook.presto.spi.type.RealType.REAL; -import static com.facebook.presto.spi.type.SmallintType.SMALLINT; import static com.facebook.presto.spi.type.TimestampType.TIMESTAMP; -import static com.facebook.presto.spi.type.TinyintType.TINYINT; import static com.facebook.presto.spi.type.VarbinaryType.VARBINARY; import static com.facebook.presto.spi.type.VarcharType.VARCHAR; import static com.facebook.presto.sql.ExpressionUtils.and; import static com.facebook.presto.sql.ExpressionUtils.or; -import static com.facebook.presto.sql.planner.LiteralInterpreter.toExpression; import static com.facebook.presto.sql.tree.BooleanLiteral.FALSE_LITERAL; import static com.facebook.presto.sql.tree.BooleanLiteral.TRUE_LITERAL; import static com.facebook.presto.sql.tree.ComparisonExpressionType.EQUAL; @@ -83,70 +78,50 @@ import static com.facebook.presto.sql.tree.ComparisonExpressionType.LESS_THAN; import static com.facebook.presto.sql.tree.ComparisonExpressionType.LESS_THAN_OR_EQUAL; import static com.facebook.presto.sql.tree.ComparisonExpressionType.NOT_EQUAL; -import static com.facebook.presto.testing.TestingConnectorSession.SESSION; import static com.facebook.presto.type.ColorType.COLOR; import static io.airlift.slice.Slices.utf8Slice; -import static java.lang.String.format; import static java.util.Collections.nCopies; -import static java.util.Objects.requireNonNull; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; -import static org.testng.Assert.fail; public class TestDomainTranslator { private static final Metadata METADATA = MetadataManager.createTestMetadataManager(); - private static final Symbol C_BIGINT = new Symbol("c_bigint"); - private static final Symbol C_DOUBLE = new Symbol("c_double"); - private static final Symbol C_VARCHAR = new Symbol("c_varchar"); - private static final Symbol C_BOOLEAN = new Symbol("c_boolean"); - private static final Symbol C_BIGINT_1 = new Symbol("c_bigint_1"); - private static final Symbol C_DOUBLE_1 = new Symbol("c_double_1"); - private static final Symbol C_VARCHAR_1 = new Symbol("c_varchar_1"); - private static final Symbol C_TIMESTAMP = new Symbol("c_timestamp"); - private static final Symbol C_DATE = new Symbol("c_date"); - private static final Symbol C_COLOR = new Symbol("c_color"); - private static final Symbol C_HYPER_LOG_LOG = new Symbol("c_hyper_log_log"); - private static final Symbol C_VARBINARY = new Symbol("c_varbinary"); - private static final Symbol C_DECIMAL_26_5 = new Symbol("c_decimal_26_5"); - private static final Symbol C_DECIMAL_23_4 = new Symbol("c_decimal_23_4"); - private static final Symbol C_INTEGER = new Symbol("c_integer"); - private static final Symbol C_CHAR = new Symbol("c_char"); - private static final Symbol C_DECIMAL_21_3 = new Symbol("c_decimal_21_3"); - private static final Symbol C_DECIMAL_12_2 = new Symbol("c_decimal_12_2"); - private static final Symbol C_DECIMAL_6_1 = new Symbol("c_decimal_6_1"); - private static final Symbol C_DECIMAL_3_0 = new Symbol("c_decimal_3_0"); - private static final Symbol C_DECIMAL_2_0 = new Symbol("c_decimal_2_0"); - private static final Symbol C_SMALLINT = new Symbol("c_smallint"); - private static final Symbol C_TINYINT = new Symbol("c_tinyint"); - private static final Symbol C_REAL = new Symbol("c_real"); + private static final Symbol A = new Symbol("a"); + private static final Symbol B = new Symbol("b"); + private static final Symbol C = new Symbol("c"); + private static final Symbol D = new Symbol("d"); + private static final Symbol E = new Symbol("e"); + private static final Symbol F = new Symbol("f"); + private static final Symbol G = new Symbol("g"); + private static final Symbol H = new Symbol("h"); + private static final Symbol I = new Symbol("i"); + private static final Symbol J = new Symbol("j"); + private static final Symbol K = new Symbol("k"); + private static final Symbol L = new Symbol("l"); + private static final Symbol M = new Symbol("m"); + private static final Symbol N = new Symbol("n"); + private static final Symbol O = new Symbol("o"); + private static final Symbol P = new Symbol("p"); private static final Map TYPES = ImmutableMap.builder() - .put(C_BIGINT, BIGINT) - .put(C_DOUBLE, DOUBLE) - .put(C_VARCHAR, VARCHAR) - .put(C_BOOLEAN, BOOLEAN) - .put(C_BIGINT_1, BIGINT) - .put(C_DOUBLE_1, DOUBLE) - .put(C_VARCHAR_1, VARCHAR) - .put(C_TIMESTAMP, TIMESTAMP) - .put(C_DATE, DATE) - .put(C_COLOR, COLOR) // Equatable, but not orderable - .put(C_HYPER_LOG_LOG, HYPER_LOG_LOG) // Not Equatable or orderable - .put(C_VARBINARY, VARBINARY) - .put(C_DECIMAL_26_5, createDecimalType(26, 5)) - .put(C_DECIMAL_23_4, createDecimalType(23, 4)) - .put(C_INTEGER, INTEGER) - .put(C_CHAR, createCharType(10)) - .put(C_DECIMAL_21_3, createDecimalType(21, 3)) - .put(C_DECIMAL_12_2, createDecimalType(12, 2)) - .put(C_DECIMAL_6_1, createDecimalType(6, 1)) - .put(C_DECIMAL_3_0, createDecimalType(3, 0)) - .put(C_DECIMAL_2_0, createDecimalType(2, 0)) - .put(C_SMALLINT, SMALLINT) - .put(C_TINYINT, TINYINT) - .put(C_REAL, REAL) + .put(A, BIGINT) + .put(B, DOUBLE) + .put(C, VARCHAR) + .put(D, BOOLEAN) + .put(E, BIGINT) + .put(F, DOUBLE) + .put(G, VARCHAR) + .put(H, TIMESTAMP) + .put(I, DATE) + .put(J, COLOR) // Equatable, but not orderable + .put(K, HYPER_LOG_LOG) // Not Equatable or orderable + .put(L, VARBINARY) + .put(M, createDecimalType(10, 5)) + .put(N, createDecimalType(4, 2)) + .put(O, INTEGER) + .put(P, createCharType(10)) .build(); private static final long TIMESTAMP_VALUE = new DateTime(2013, 3, 30, 1, 5, 0, 0, DateTimeZone.UTC).getMillis(); @@ -179,17 +154,17 @@ public void testRoundTrip() throws Exception { TupleDomain tupleDomain = withColumnDomains(ImmutableMap.builder() - .put(C_BIGINT, Domain.singleValue(BIGINT, 1L)) - .put(C_DOUBLE, Domain.onlyNull(DOUBLE)) - .put(C_VARCHAR, Domain.notNull(VARCHAR)) - .put(C_BOOLEAN, Domain.singleValue(BOOLEAN, true)) - .put(C_BIGINT_1, Domain.singleValue(BIGINT, 2L)) - .put(C_DOUBLE_1, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(DOUBLE, 1.1), Range.equal(DOUBLE, 2.0), Range.range(DOUBLE, 3.0, false, 3.5, true)), true)) - .put(C_VARCHAR_1, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(VARCHAR, utf8Slice("2013-01-01")), Range.greaterThan(VARCHAR, utf8Slice("2013-10-01"))), false)) - .put(C_TIMESTAMP, Domain.singleValue(TIMESTAMP, TIMESTAMP_VALUE)) - .put(C_DATE, Domain.singleValue(DATE, DATE_VALUE)) - .put(C_COLOR, Domain.singleValue(COLOR, COLOR_VALUE_1)) - .put(C_HYPER_LOG_LOG, Domain.notNull(HYPER_LOG_LOG)) + .put(A, Domain.singleValue(BIGINT, 1L)) + .put(B, Domain.onlyNull(DOUBLE)) + .put(C, Domain.notNull(VARCHAR)) + .put(D, Domain.singleValue(BOOLEAN, true)) + .put(E, Domain.singleValue(BIGINT, 2L)) + .put(F, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(DOUBLE, 1.1), Range.equal(DOUBLE, 2.0), Range.range(DOUBLE, 3.0, false, 3.5, true)), true)) + .put(G, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(VARCHAR, utf8Slice("2013-01-01")), Range.greaterThan(VARCHAR, utf8Slice("2013-10-01"))), false)) + .put(H, Domain.singleValue(TIMESTAMP, TIMESTAMP_VALUE)) + .put(I, Domain.singleValue(DATE, DATE_VALUE)) + .put(J, Domain.singleValue(COLOR, COLOR_VALUE_1)) + .put(K, Domain.notNull(HYPER_LOG_LOG)) .build()); ExtractionResult result = fromPredicate(toPredicate(tupleDomain)); @@ -206,8 +181,8 @@ public void testInOptimization() .subtract(ValueSet.ofRanges( Range.equal(BIGINT, 1L), Range.equal(BIGINT, 2L), Range.equal(BIGINT, 3L))), false); - TupleDomain tupleDomain = withColumnDomains(ImmutableMap.builder().put(C_BIGINT, testDomain).build()); - assertEquals(toPredicate(tupleDomain), not(in(C_BIGINT, ImmutableList.of(1L, 2L, 3L)))); + TupleDomain tupleDomain = withColumnDomains(ImmutableMap.builder().put(A, testDomain).build()); + assertEquals(toPredicate(tupleDomain), not(in(A, ImmutableList.of(1L, 2L, 3L)))); testDomain = Domain.create( ValueSet.ofRanges( @@ -215,8 +190,8 @@ public void testInOptimization() ValueSet.all(BIGINT) .subtract(ValueSet.ofRanges(Range.equal(BIGINT, 1L), Range.equal(BIGINT, 2L), Range.equal(BIGINT, 3L)))), false); - tupleDomain = withColumnDomains(ImmutableMap.builder().put(C_BIGINT, testDomain).build()); - assertEquals(toPredicate(tupleDomain), and(lessThan(C_BIGINT, bigintLiteral(4L)), not(in(C_BIGINT, ImmutableList.of(1L, 2L, 3L))))); + tupleDomain = withColumnDomains(ImmutableMap.builder().put(A, testDomain).build()); + assertEquals(toPredicate(tupleDomain), and(lessThan(A, bigintLiteral(4L)), not(in(A, ImmutableList.of(1L, 2L, 3L))))); testDomain = Domain.create(ValueSet.ofRanges( Range.range(BIGINT, 1L, true, 3L, true), @@ -224,9 +199,9 @@ public void testInOptimization() Range.range(BIGINT, 9L, true, 11L, true)), false); - tupleDomain = withColumnDomains(ImmutableMap.builder().put(C_BIGINT, testDomain).build()); + tupleDomain = withColumnDomains(ImmutableMap.builder().put(A, testDomain).build()); assertEquals(toPredicate(tupleDomain), - or(between(C_BIGINT, bigintLiteral(1L), bigintLiteral(3L)), (between(C_BIGINT, bigintLiteral(5L), bigintLiteral(7L))), (between(C_BIGINT, bigintLiteral(9L), bigintLiteral(11L))))); + or(between(A, bigintLiteral(1L), bigintLiteral(3L)), (between(A, bigintLiteral(5L), bigintLiteral(7L))), (between(A, bigintLiteral(9L), bigintLiteral(11L))))); testDomain = Domain.create( ValueSet.ofRanges( @@ -235,8 +210,8 @@ public void testInOptimization() .subtract(ValueSet.ofRanges(Range.equal(BIGINT, 1L), Range.equal(BIGINT, 2L), Range.equal(BIGINT, 3L)))) .union(ValueSet.ofRanges(Range.range(BIGINT, 7L, true, 9L, true))), false); - tupleDomain = withColumnDomains(ImmutableMap.builder().put(C_BIGINT, testDomain).build()); - assertEquals(toPredicate(tupleDomain), or(and(lessThan(C_BIGINT, bigintLiteral(4L)), not(in(C_BIGINT, ImmutableList.of(1L, 2L, 3L)))), between(C_BIGINT, bigintLiteral(7L), bigintLiteral(9L)))); + tupleDomain = withColumnDomains(ImmutableMap.builder().put(A, testDomain).build()); + assertEquals(toPredicate(tupleDomain), or(and(lessThan(A, bigintLiteral(4L)), not(in(A, ImmutableList.of(1L, 2L, 3L)))), between(A, bigintLiteral(7L), bigintLiteral(9L)))); testDomain = Domain.create( ValueSet.ofRanges(Range.lessThan(BIGINT, 4L)) @@ -244,11 +219,11 @@ public void testInOptimization() .subtract(ValueSet.ofRanges(Range.equal(BIGINT, 1L), Range.equal(BIGINT, 2L), Range.equal(BIGINT, 3L)))) .union(ValueSet.ofRanges(Range.range(BIGINT, 7L, false, 9L, false), Range.range(BIGINT, 11L, false, 13L, false))), false); - tupleDomain = withColumnDomains(ImmutableMap.builder().put(C_BIGINT, testDomain).build()); + tupleDomain = withColumnDomains(ImmutableMap.builder().put(A, testDomain).build()); assertEquals(toPredicate(tupleDomain), or( - and(lessThan(C_BIGINT, bigintLiteral(4L)), not(in(C_BIGINT, ImmutableList.of(1L, 2L, 3L)))), - and(greaterThan(C_BIGINT, bigintLiteral(7L)), lessThan(C_BIGINT, bigintLiteral(9L))), - and(greaterThan(C_BIGINT, bigintLiteral(11L)), lessThan(C_BIGINT, bigintLiteral(13L))))); + and(lessThan(A, bigintLiteral(4L)), not(in(A, ImmutableList.of(1L, 2L, 3L)))), + and(greaterThan(A, bigintLiteral(7L)), lessThan(A, bigintLiteral(9L))), + and(greaterThan(A, bigintLiteral(11L)), lessThan(A, bigintLiteral(13L))))); } @Test @@ -256,10 +231,10 @@ public void testToPredicateNone() throws Exception { TupleDomain tupleDomain = withColumnDomains(ImmutableMap.builder() - .put(C_BIGINT, Domain.singleValue(BIGINT, 1L)) - .put(C_DOUBLE, Domain.onlyNull(DOUBLE)) - .put(C_VARCHAR, Domain.notNull(VARCHAR)) - .put(C_BOOLEAN, Domain.none(BOOLEAN)) + .put(A, Domain.singleValue(BIGINT, 1L)) + .put(B, Domain.onlyNull(DOUBLE)) + .put(C, Domain.notNull(VARCHAR)) + .put(D, Domain.none(BOOLEAN)) .build()); assertEquals(toPredicate(tupleDomain), FALSE_LITERAL); @@ -270,18 +245,18 @@ public void testToPredicateAllIgnored() throws Exception { TupleDomain tupleDomain = withColumnDomains(ImmutableMap.builder() - .put(C_BIGINT, Domain.singleValue(BIGINT, 1L)) - .put(C_DOUBLE, Domain.onlyNull(DOUBLE)) - .put(C_VARCHAR, Domain.notNull(VARCHAR)) - .put(C_BOOLEAN, Domain.all(BOOLEAN)) + .put(A, Domain.singleValue(BIGINT, 1L)) + .put(B, Domain.onlyNull(DOUBLE)) + .put(C, Domain.notNull(VARCHAR)) + .put(D, Domain.all(BOOLEAN)) .build()); ExtractionResult result = fromPredicate(toPredicate(tupleDomain)); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.builder() - .put(C_BIGINT, Domain.singleValue(BIGINT, 1L)) - .put(C_DOUBLE, Domain.onlyNull(DOUBLE)) - .put(C_VARCHAR, Domain.notNull(VARCHAR)) + .put(A, Domain.singleValue(BIGINT, 1L)) + .put(B, Domain.onlyNull(DOUBLE)) + .put(C, Domain.notNull(VARCHAR)) .build())); } @@ -291,67 +266,67 @@ public void testToPredicate() { TupleDomain tupleDomain; - tupleDomain = withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.notNull(BIGINT))); - assertEquals(toPredicate(tupleDomain), isNotNull(C_BIGINT)); + tupleDomain = withColumnDomains(ImmutableMap.of(A, Domain.notNull(BIGINT))); + assertEquals(toPredicate(tupleDomain), isNotNull(A)); - tupleDomain = withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.onlyNull(BIGINT))); - assertEquals(toPredicate(tupleDomain), isNull(C_BIGINT)); + tupleDomain = withColumnDomains(ImmutableMap.of(A, Domain.onlyNull(BIGINT))); + assertEquals(toPredicate(tupleDomain), isNull(A)); - tupleDomain = withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.none(BIGINT))); + tupleDomain = withColumnDomains(ImmutableMap.of(A, Domain.none(BIGINT))); assertEquals(toPredicate(tupleDomain), FALSE_LITERAL); - tupleDomain = withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.all(BIGINT))); + tupleDomain = withColumnDomains(ImmutableMap.of(A, Domain.all(BIGINT))); assertEquals(toPredicate(tupleDomain), TRUE_LITERAL); - tupleDomain = withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 1L)), false))); - assertEquals(toPredicate(tupleDomain), greaterThan(C_BIGINT, bigintLiteral(1L))); + tupleDomain = withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 1L)), false))); + assertEquals(toPredicate(tupleDomain), greaterThan(A, bigintLiteral(1L))); - tupleDomain = withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.greaterThanOrEqual(BIGINT, 1L)), false))); - assertEquals(toPredicate(tupleDomain), greaterThanOrEqual(C_BIGINT, bigintLiteral(1L))); + tupleDomain = withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.greaterThanOrEqual(BIGINT, 1L)), false))); + assertEquals(toPredicate(tupleDomain), greaterThanOrEqual(A, bigintLiteral(1L))); - tupleDomain = withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 1L)), false))); - assertEquals(toPredicate(tupleDomain), lessThan(C_BIGINT, bigintLiteral(1L))); + tupleDomain = withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 1L)), false))); + assertEquals(toPredicate(tupleDomain), lessThan(A, bigintLiteral(1L))); - tupleDomain = withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.range(BIGINT, 0L, false, 1L, true)), false))); - assertEquals(toPredicate(tupleDomain), and(greaterThan(C_BIGINT, bigintLiteral(0L)), lessThanOrEqual(C_BIGINT, bigintLiteral(1L)))); + tupleDomain = withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.range(BIGINT, 0L, false, 1L, true)), false))); + assertEquals(toPredicate(tupleDomain), and(greaterThan(A, bigintLiteral(0L)), lessThanOrEqual(A, bigintLiteral(1L)))); - tupleDomain = withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(BIGINT, 1L)), false))); - assertEquals(toPredicate(tupleDomain), lessThanOrEqual(C_BIGINT, bigintLiteral(1L))); + tupleDomain = withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(BIGINT, 1L)), false))); + assertEquals(toPredicate(tupleDomain), lessThanOrEqual(A, bigintLiteral(1L))); - tupleDomain = withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.singleValue(BIGINT, 1L))); - assertEquals(toPredicate(tupleDomain), equal(C_BIGINT, bigintLiteral(1L))); + tupleDomain = withColumnDomains(ImmutableMap.of(A, Domain.singleValue(BIGINT, 1L))); + assertEquals(toPredicate(tupleDomain), equal(A, bigintLiteral(1L))); - tupleDomain = withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 1L), Range.equal(BIGINT, 2L)), false))); - assertEquals(toPredicate(tupleDomain), in(C_BIGINT, ImmutableList.of(1L, 2L))); + tupleDomain = withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 1L), Range.equal(BIGINT, 2L)), false))); + assertEquals(toPredicate(tupleDomain), in(A, ImmutableList.of(1L, 2L))); - tupleDomain = withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 1L)), true))); - assertEquals(toPredicate(tupleDomain), or(lessThan(C_BIGINT, bigintLiteral(1L)), isNull(C_BIGINT))); + tupleDomain = withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 1L)), true))); + assertEquals(toPredicate(tupleDomain), or(lessThan(A, bigintLiteral(1L)), isNull(A))); - tupleDomain = withColumnDomains(ImmutableMap.of(C_COLOR, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1), true))); - assertEquals(toPredicate(tupleDomain), or(equal(C_COLOR, colorLiteral(COLOR_VALUE_1)), isNull(C_COLOR))); + tupleDomain = withColumnDomains(ImmutableMap.of(J, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1), true))); + assertEquals(toPredicate(tupleDomain), or(equal(J, colorLiteral(COLOR_VALUE_1)), isNull(J))); - tupleDomain = withColumnDomains(ImmutableMap.of(C_COLOR, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1).complement(), true))); - assertEquals(toPredicate(tupleDomain), or(not(equal(C_COLOR, colorLiteral(COLOR_VALUE_1))), isNull(C_COLOR))); + tupleDomain = withColumnDomains(ImmutableMap.of(J, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1).complement(), true))); + assertEquals(toPredicate(tupleDomain), or(not(equal(J, colorLiteral(COLOR_VALUE_1))), isNull(J))); - tupleDomain = withColumnDomains(ImmutableMap.of(C_HYPER_LOG_LOG, Domain.onlyNull(HYPER_LOG_LOG))); - assertEquals(toPredicate(tupleDomain), isNull(C_HYPER_LOG_LOG)); + tupleDomain = withColumnDomains(ImmutableMap.of(K, Domain.onlyNull(HYPER_LOG_LOG))); + assertEquals(toPredicate(tupleDomain), isNull(K)); - tupleDomain = withColumnDomains(ImmutableMap.of(C_HYPER_LOG_LOG, Domain.notNull(HYPER_LOG_LOG))); - assertEquals(toPredicate(tupleDomain), isNotNull(C_HYPER_LOG_LOG)); + tupleDomain = withColumnDomains(ImmutableMap.of(K, Domain.notNull(HYPER_LOG_LOG))); + assertEquals(toPredicate(tupleDomain), isNotNull(K)); } @Test public void testFromUnknownPredicate() throws Exception { - ExtractionResult result = fromPredicate(unprocessableExpression1(C_BIGINT)); + ExtractionResult result = fromPredicate(unprocessableExpression1(A)); assertTrue(result.getTupleDomain().isAll()); - assertEquals(result.getRemainingExpression(), unprocessableExpression1(C_BIGINT)); + assertEquals(result.getRemainingExpression(), unprocessableExpression1(A)); // Test the complement - result = fromPredicate(not(unprocessableExpression1(C_BIGINT))); + result = fromPredicate(not(unprocessableExpression1(A))); assertTrue(result.getTupleDomain().isAll()); - assertEquals(result.getRemainingExpression(), not(unprocessableExpression1(C_BIGINT))); + assertEquals(result.getRemainingExpression(), not(unprocessableExpression1(A))); } @Test @@ -359,26 +334,26 @@ public void testFromAndPredicate() throws Exception { Expression originalPredicate = and( - and(greaterThan(C_BIGINT, bigintLiteral(1L)), unprocessableExpression1(C_BIGINT)), - and(lessThan(C_BIGINT, bigintLiteral(5L)), unprocessableExpression2(C_BIGINT))); + and(greaterThan(A, bigintLiteral(1L)), unprocessableExpression1(A)), + and(lessThan(A, bigintLiteral(5L)), unprocessableExpression2(A))); ExtractionResult result = fromPredicate(originalPredicate); - assertEquals(result.getRemainingExpression(), and(unprocessableExpression1(C_BIGINT), unprocessableExpression2(C_BIGINT))); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.range(BIGINT, 1L, false, 5L, false)), false)))); + assertEquals(result.getRemainingExpression(), and(unprocessableExpression1(A), unprocessableExpression2(A))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.range(BIGINT, 1L, false, 5L, false)), false)))); // Test complements originalPredicate = not(and( - and(greaterThan(C_BIGINT, bigintLiteral(1L)), unprocessableExpression1(C_BIGINT)), - and(lessThan(C_BIGINT, bigintLiteral(5L)), unprocessableExpression2(C_BIGINT)))); + and(greaterThan(A, bigintLiteral(1L)), unprocessableExpression1(A)), + and(lessThan(A, bigintLiteral(5L)), unprocessableExpression2(A)))); result = fromPredicate(originalPredicate); assertEquals(result.getRemainingExpression(), originalPredicate); assertTrue(result.getTupleDomain().isAll()); originalPredicate = not(and( - not(and(greaterThan(C_BIGINT, bigintLiteral(1L)), unprocessableExpression1(C_BIGINT))), - not(and(lessThan(C_BIGINT, bigintLiteral(5L)), unprocessableExpression2(C_BIGINT))))); + not(and(greaterThan(A, bigintLiteral(1L)), unprocessableExpression1(A))), + not(and(lessThan(A, bigintLiteral(5L)), unprocessableExpression2(A))))); result = fromPredicate(originalPredicate); assertEquals(result.getRemainingExpression(), originalPredicate); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.notNull(BIGINT)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.notNull(BIGINT)))); } @Test @@ -386,82 +361,82 @@ public void testFromOrPredicate() throws Exception { Expression originalPredicate = or( - and(greaterThan(C_BIGINT, bigintLiteral(1L)), unprocessableExpression1(C_BIGINT)), - and(lessThan(C_BIGINT, bigintLiteral(5L)), unprocessableExpression2(C_BIGINT))); + and(greaterThan(A, bigintLiteral(1L)), unprocessableExpression1(A)), + and(lessThan(A, bigintLiteral(5L)), unprocessableExpression2(A))); ExtractionResult result = fromPredicate(originalPredicate); assertEquals(result.getRemainingExpression(), originalPredicate); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.notNull(BIGINT)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.notNull(BIGINT)))); originalPredicate = or( - and(equal(C_BIGINT, bigintLiteral(1L)), unprocessableExpression1(C_BIGINT)), - and(equal(C_BIGINT, bigintLiteral(2L)), unprocessableExpression2(C_BIGINT))); + and(equal(A, bigintLiteral(1L)), unprocessableExpression1(A)), + and(equal(A, bigintLiteral(2L)), unprocessableExpression2(A))); result = fromPredicate(originalPredicate); assertEquals(result.getRemainingExpression(), originalPredicate); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 1L), Range.equal(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 1L), Range.equal(BIGINT, 2L)), false)))); // Same unprocessableExpression means that we can do more extraction // If both sides are operating on the same single symbol originalPredicate = or( - and(equal(C_BIGINT, bigintLiteral(1L)), unprocessableExpression1(C_BIGINT)), - and(equal(C_BIGINT, bigintLiteral(2L)), unprocessableExpression1(C_BIGINT))); + and(equal(A, bigintLiteral(1L)), unprocessableExpression1(A)), + and(equal(A, bigintLiteral(2L)), unprocessableExpression1(A))); result = fromPredicate(originalPredicate); - assertEquals(result.getRemainingExpression(), unprocessableExpression1(C_BIGINT)); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 1L), Range.equal(BIGINT, 2L)), false)))); + assertEquals(result.getRemainingExpression(), unprocessableExpression1(A)); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 1L), Range.equal(BIGINT, 2L)), false)))); // And not if they have different symbols originalPredicate = or( - and(equal(C_BIGINT, bigintLiteral(1L)), unprocessableExpression1(C_BIGINT)), - and(equal(C_DOUBLE, doubleLiteral(2.0)), unprocessableExpression1(C_BIGINT))); + and(equal(A, bigintLiteral(1L)), unprocessableExpression1(A)), + and(equal(B, doubleLiteral(2.0)), unprocessableExpression1(A))); result = fromPredicate(originalPredicate); assertEquals(result.getRemainingExpression(), originalPredicate); assertTrue(result.getTupleDomain().isAll()); // We can make another optimization if one side is the super set of the other side originalPredicate = or( - and(greaterThan(C_BIGINT, bigintLiteral(1L)), greaterThan(C_DOUBLE, doubleLiteral(1.0)), unprocessableExpression1(C_BIGINT)), - and(greaterThan(C_BIGINT, bigintLiteral(2L)), greaterThan(C_DOUBLE, doubleLiteral(2.0)), unprocessableExpression1(C_BIGINT))); + and(greaterThan(A, bigintLiteral(1L)), greaterThan(B, doubleLiteral(1.0)), unprocessableExpression1(A)), + and(greaterThan(A, bigintLiteral(2L)), greaterThan(B, doubleLiteral(2.0)), unprocessableExpression1(A))); result = fromPredicate(originalPredicate); - assertEquals(result.getRemainingExpression(), unprocessableExpression1(C_BIGINT)); + assertEquals(result.getRemainingExpression(), unprocessableExpression1(A)); assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of( - C_BIGINT, Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 1L)), false), - C_DOUBLE, Domain.create(ValueSet.ofRanges(Range.greaterThan(DOUBLE, 1.0)), false)))); + A, Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 1L)), false), + B, Domain.create(ValueSet.ofRanges(Range.greaterThan(DOUBLE, 1.0)), false)))); // We can't make those inferences if the unprocessableExpressions are non-deterministic originalPredicate = or( - and(equal(C_BIGINT, bigintLiteral(1L)), randPredicate(C_BIGINT)), - and(equal(C_BIGINT, bigintLiteral(2L)), randPredicate(C_BIGINT))); + and(equal(A, bigintLiteral(1L)), randPredicate(A)), + and(equal(A, bigintLiteral(2L)), randPredicate(A))); result = fromPredicate(originalPredicate); assertEquals(result.getRemainingExpression(), originalPredicate); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 1L), Range.equal(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 1L), Range.equal(BIGINT, 2L)), false)))); // Test complements originalPredicate = not(or( - and(greaterThan(C_BIGINT, bigintLiteral(1L)), unprocessableExpression1(C_BIGINT)), - and(lessThan(C_BIGINT, bigintLiteral(5L)), unprocessableExpression2(C_BIGINT)))); + and(greaterThan(A, bigintLiteral(1L)), unprocessableExpression1(A)), + and(lessThan(A, bigintLiteral(5L)), unprocessableExpression2(A)))); result = fromPredicate(originalPredicate); assertEquals(result.getRemainingExpression(), and( - not(and(greaterThan(C_BIGINT, bigintLiteral(1L)), unprocessableExpression1(C_BIGINT))), - not(and(lessThan(C_BIGINT, bigintLiteral(5L)), unprocessableExpression2(C_BIGINT))))); + not(and(greaterThan(A, bigintLiteral(1L)), unprocessableExpression1(A))), + not(and(lessThan(A, bigintLiteral(5L)), unprocessableExpression2(A))))); assertTrue(result.getTupleDomain().isAll()); originalPredicate = not(or( - not(and(greaterThan(C_BIGINT, bigintLiteral(1L)), unprocessableExpression1(C_BIGINT))), - not(and(lessThan(C_BIGINT, bigintLiteral(5L)), unprocessableExpression2(C_BIGINT))))); + not(and(greaterThan(A, bigintLiteral(1L)), unprocessableExpression1(A))), + not(and(lessThan(A, bigintLiteral(5L)), unprocessableExpression2(A))))); result = fromPredicate(originalPredicate); - assertEquals(result.getRemainingExpression(), and(unprocessableExpression1(C_BIGINT), unprocessableExpression2(C_BIGINT))); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.range(BIGINT, 1L, false, 5L, false)), false)))); + assertEquals(result.getRemainingExpression(), and(unprocessableExpression1(A), unprocessableExpression2(A))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.range(BIGINT, 1L, false, 5L, false)), false)))); } @Test public void testFromNotPredicate() throws Exception { - Expression originalPredicate = not(and(equal(C_BIGINT, bigintLiteral(1L)), unprocessableExpression1(C_BIGINT))); + Expression originalPredicate = not(and(equal(A, bigintLiteral(1L)), unprocessableExpression1(A))); ExtractionResult result = fromPredicate(originalPredicate); assertEquals(result.getRemainingExpression(), originalPredicate); assertTrue(result.getTupleDomain().isAll()); - originalPredicate = not(unprocessableExpression1(C_BIGINT)); + originalPredicate = not(unprocessableExpression1(A)); result = fromPredicate(originalPredicate); assertEquals(result.getRemainingExpression(), originalPredicate); assertTrue(result.getTupleDomain().isAll()); @@ -471,10 +446,10 @@ public void testFromNotPredicate() assertEquals(result.getRemainingExpression(), TRUE_LITERAL); assertTrue(result.getTupleDomain().isNone()); - originalPredicate = not(equal(C_BIGINT, bigintLiteral(1L))); + originalPredicate = not(equal(A, bigintLiteral(1L))); result = fromPredicate(originalPredicate); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 1L), Range.greaterThan(BIGINT, 1L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 1L), Range.greaterThan(BIGINT, 1L)), false)))); } @Test @@ -482,123 +457,133 @@ public void testFromUnprocessableComparison() throws Exception { // If it is not a simple comparison, we should not try to process it - Expression predicate = comparison(GREATER_THAN, unprocessableExpression1(C_BIGINT), unprocessableExpression2(C_BIGINT)); + Expression predicate = comparison(GREATER_THAN, unprocessableExpression1(A), unprocessableExpression2(A)); ExtractionResult result = fromPredicate(predicate); assertEquals(result.getRemainingExpression(), predicate); assertTrue(result.getTupleDomain().isAll()); // Complement - predicate = not(comparison(GREATER_THAN, unprocessableExpression1(C_BIGINT), unprocessableExpression2(C_BIGINT))); + predicate = not(comparison(GREATER_THAN, unprocessableExpression1(A), unprocessableExpression2(A))); result = fromPredicate(predicate); assertEquals(result.getRemainingExpression(), predicate); assertTrue(result.getTupleDomain().isAll()); } + @Test + public void testFromDecimalComparison() + throws Exception + { + Expression predicate = greaterThan(M, decimalLiteral("12.345")); + ExtractionResult result = fromPredicate(predicate); + assertEquals(result.getRemainingExpression(), TRUE_LITERAL); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(M, Domain.create(ValueSet.ofRanges(Range.greaterThan(createDecimalType(10, 5), 1234500L)), false)))); + } + @Test public void testFromBasicComparisons() throws Exception { // Test out the extraction of all basic comparisons - Expression originalExpression = greaterThan(C_BIGINT, bigintLiteral(2L)); + Expression originalExpression = greaterThan(A, bigintLiteral(2L)); ExtractionResult result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 2L)), false)))); - originalExpression = greaterThanOrEqual(C_BIGINT, bigintLiteral(2L)); + originalExpression = greaterThanOrEqual(A, bigintLiteral(2L)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.greaterThanOrEqual(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.greaterThanOrEqual(BIGINT, 2L)), false)))); - originalExpression = lessThan(C_BIGINT, bigintLiteral(2L)); + originalExpression = lessThan(A, bigintLiteral(2L)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L)), false)))); - originalExpression = lessThanOrEqual(C_BIGINT, bigintLiteral(2L)); + originalExpression = lessThanOrEqual(A, bigintLiteral(2L)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(BIGINT, 2L)), false)))); - originalExpression = equal(C_BIGINT, bigintLiteral(2L)); + originalExpression = equal(A, bigintLiteral(2L)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 2L)), false)))); - originalExpression = notEqual(C_BIGINT, bigintLiteral(2L)); + originalExpression = notEqual(A, bigintLiteral(2L)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L), Range.greaterThan(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L), Range.greaterThan(BIGINT, 2L)), false)))); - originalExpression = isDistinctFrom(C_BIGINT, bigintLiteral(2L)); + originalExpression = isDistinctFrom(A, bigintLiteral(2L)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L), Range.greaterThan(BIGINT, 2L)), true)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L), Range.greaterThan(BIGINT, 2L)), true)))); - originalExpression = equal(C_COLOR, colorLiteral(COLOR_VALUE_1)); + originalExpression = equal(J, colorLiteral(COLOR_VALUE_1)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_COLOR, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(J, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1), false)))); - originalExpression = in(C_COLOR, ImmutableList.of(colorLiteral(COLOR_VALUE_1), colorLiteral(COLOR_VALUE_2))); + originalExpression = in(J, ImmutableList.of(colorLiteral(COLOR_VALUE_1), colorLiteral(COLOR_VALUE_2))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_COLOR, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1, COLOR_VALUE_2), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(J, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1, COLOR_VALUE_2), false)))); - originalExpression = isDistinctFrom(C_COLOR, colorLiteral(COLOR_VALUE_1)); + originalExpression = isDistinctFrom(J, colorLiteral(COLOR_VALUE_1)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_COLOR, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1).complement(), true)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(J, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1).complement(), true)))); // Test complement - originalExpression = not(greaterThan(C_BIGINT, bigintLiteral(2L))); + originalExpression = not(greaterThan(A, bigintLiteral(2L))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(BIGINT, 2L)), false)))); - originalExpression = not(greaterThanOrEqual(C_BIGINT, bigintLiteral(2L))); + originalExpression = not(greaterThanOrEqual(A, bigintLiteral(2L))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L)), false)))); - originalExpression = not(lessThan(C_BIGINT, bigintLiteral(2L))); + originalExpression = not(lessThan(A, bigintLiteral(2L))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.greaterThanOrEqual(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.greaterThanOrEqual(BIGINT, 2L)), false)))); - originalExpression = not(lessThanOrEqual(C_BIGINT, bigintLiteral(2L))); + originalExpression = not(lessThanOrEqual(A, bigintLiteral(2L))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 2L)), false)))); - originalExpression = not(equal(C_BIGINT, bigintLiteral(2L))); + originalExpression = not(equal(A, bigintLiteral(2L))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L), Range.greaterThan(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L), Range.greaterThan(BIGINT, 2L)), false)))); - originalExpression = not(notEqual(C_BIGINT, bigintLiteral(2L))); + originalExpression = not(notEqual(A, bigintLiteral(2L))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 2L)), false)))); - originalExpression = not(isDistinctFrom(C_BIGINT, bigintLiteral(2L))); + originalExpression = not(isDistinctFrom(A, bigintLiteral(2L))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 2L)), false)))); - originalExpression = not(equal(C_COLOR, colorLiteral(COLOR_VALUE_1))); + originalExpression = not(equal(J, colorLiteral(COLOR_VALUE_1))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_COLOR, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1).complement(), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(J, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1).complement(), false)))); - originalExpression = not(in(C_COLOR, ImmutableList.of(colorLiteral(COLOR_VALUE_1), colorLiteral(COLOR_VALUE_2)))); + originalExpression = not(in(J, ImmutableList.of(colorLiteral(COLOR_VALUE_1), colorLiteral(COLOR_VALUE_2)))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_COLOR, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1, COLOR_VALUE_2).complement(), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(J, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1, COLOR_VALUE_2).complement(), false)))); - originalExpression = not(isDistinctFrom(C_COLOR, colorLiteral(COLOR_VALUE_1))); + originalExpression = not(isDistinctFrom(J, colorLiteral(COLOR_VALUE_1))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_COLOR, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(J, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1), false)))); } @Test @@ -606,60 +591,60 @@ public void testFromFlippedBasicComparisons() throws Exception { // Test out the extraction of all basic comparisons where the reference literal ordering is flipped - ComparisonExpression originalExpression = comparison(GREATER_THAN, bigintLiteral(2L), C_BIGINT.toSymbolReference()); + ComparisonExpression originalExpression = comparison(GREATER_THAN, bigintLiteral(2L), A.toSymbolReference()); ExtractionResult result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L)), false)))); - originalExpression = comparison(GREATER_THAN_OR_EQUAL, bigintLiteral(2L), C_BIGINT.toSymbolReference()); + originalExpression = comparison(GREATER_THAN_OR_EQUAL, bigintLiteral(2L), A.toSymbolReference()); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(BIGINT, 2L)), false)))); - originalExpression = comparison(LESS_THAN, bigintLiteral(2L), C_BIGINT.toSymbolReference()); + originalExpression = comparison(LESS_THAN, bigintLiteral(2L), A.toSymbolReference()); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 2L)), false)))); - originalExpression = comparison(LESS_THAN_OR_EQUAL, bigintLiteral(2L), C_BIGINT.toSymbolReference()); + originalExpression = comparison(LESS_THAN_OR_EQUAL, bigintLiteral(2L), A.toSymbolReference()); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.greaterThanOrEqual(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.greaterThanOrEqual(BIGINT, 2L)), false)))); - originalExpression = comparison(EQUAL, bigintLiteral(2L), C_BIGINT.toSymbolReference()); + originalExpression = comparison(EQUAL, bigintLiteral(2L), A.toSymbolReference()); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 2L)), false)))); - originalExpression = comparison(EQUAL, colorLiteral(COLOR_VALUE_1), C_COLOR.toSymbolReference()); + originalExpression = comparison(EQUAL, colorLiteral(COLOR_VALUE_1), J.toSymbolReference()); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_COLOR, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(J, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1), false)))); - originalExpression = comparison(NOT_EQUAL, bigintLiteral(2L), C_BIGINT.toSymbolReference()); + originalExpression = comparison(NOT_EQUAL, bigintLiteral(2L), A.toSymbolReference()); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L), Range.greaterThan(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L), Range.greaterThan(BIGINT, 2L)), false)))); - originalExpression = comparison(NOT_EQUAL, colorLiteral(COLOR_VALUE_1), C_COLOR.toSymbolReference()); + originalExpression = comparison(NOT_EQUAL, colorLiteral(COLOR_VALUE_1), J.toSymbolReference()); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_COLOR, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1).complement(), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(J, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1).complement(), false)))); - originalExpression = comparison(IS_DISTINCT_FROM, bigintLiteral(2L), C_BIGINT.toSymbolReference()); + originalExpression = comparison(IS_DISTINCT_FROM, bigintLiteral(2L), A.toSymbolReference()); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L), Range.greaterThan(BIGINT, 2L)), true)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L), Range.greaterThan(BIGINT, 2L)), true)))); - originalExpression = comparison(IS_DISTINCT_FROM, colorLiteral(COLOR_VALUE_1), C_COLOR.toSymbolReference()); + originalExpression = comparison(IS_DISTINCT_FROM, colorLiteral(COLOR_VALUE_1), J.toSymbolReference()); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_COLOR, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1).complement(), true)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(J, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1).complement(), true)))); - originalExpression = comparison(IS_DISTINCT_FROM, nullLiteral(), C_BIGINT.toSymbolReference()); + originalExpression = comparison(IS_DISTINCT_FROM, nullLiteral(), A.toSymbolReference()); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.notNull(BIGINT)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.notNull(BIGINT)))); } @Test @@ -667,111 +652,111 @@ public void testFromBasicComparisonsWithNulls() throws Exception { // Test out the extraction of all basic comparisons with null literals - Expression originalExpression = greaterThan(C_BIGINT, nullLiteral()); + Expression originalExpression = greaterThan(A, nullLiteral()); ExtractionResult result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); assertTrue(result.getTupleDomain().isNone()); - originalExpression = greaterThan(C_VARCHAR, new Cast(nullLiteral(), StandardTypes.VARCHAR)); + originalExpression = greaterThan(C, new Cast(nullLiteral(), StandardTypes.VARCHAR)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_VARCHAR, Domain.create(ValueSet.none(VARCHAR), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C, Domain.create(ValueSet.none(VARCHAR), false)))); - originalExpression = greaterThanOrEqual(C_BIGINT, nullLiteral()); + originalExpression = greaterThanOrEqual(A, nullLiteral()); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); assertTrue(result.getTupleDomain().isNone()); - originalExpression = lessThan(C_BIGINT, nullLiteral()); + originalExpression = lessThan(A, nullLiteral()); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); assertTrue(result.getTupleDomain().isNone()); - originalExpression = lessThanOrEqual(C_BIGINT, nullLiteral()); + originalExpression = lessThanOrEqual(A, nullLiteral()); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); assertTrue(result.getTupleDomain().isNone()); - originalExpression = equal(C_BIGINT, nullLiteral()); + originalExpression = equal(A, nullLiteral()); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); assertTrue(result.getTupleDomain().isNone()); - originalExpression = equal(C_COLOR, nullLiteral()); + originalExpression = equal(J, nullLiteral()); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); assertTrue(result.getTupleDomain().isNone()); - originalExpression = notEqual(C_BIGINT, nullLiteral()); + originalExpression = notEqual(A, nullLiteral()); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); assertTrue(result.getTupleDomain().isNone()); - originalExpression = notEqual(C_COLOR, nullLiteral()); + originalExpression = notEqual(J, nullLiteral()); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); assertTrue(result.getTupleDomain().isNone()); - originalExpression = isDistinctFrom(C_BIGINT, nullLiteral()); + originalExpression = isDistinctFrom(A, nullLiteral()); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.notNull(BIGINT)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.notNull(BIGINT)))); - originalExpression = isDistinctFrom(C_COLOR, nullLiteral()); + originalExpression = isDistinctFrom(J, nullLiteral()); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_COLOR, Domain.notNull(COLOR)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(J, Domain.notNull(COLOR)))); // Test complements - originalExpression = not(greaterThan(C_BIGINT, nullLiteral())); + originalExpression = not(greaterThan(A, nullLiteral())); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); assertTrue(result.getTupleDomain().isNone()); - originalExpression = not(greaterThanOrEqual(C_BIGINT, nullLiteral())); + originalExpression = not(greaterThanOrEqual(A, nullLiteral())); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); assertTrue(result.getTupleDomain().isNone()); - originalExpression = not(lessThan(C_BIGINT, nullLiteral())); + originalExpression = not(lessThan(A, nullLiteral())); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); assertTrue(result.getTupleDomain().isNone()); - originalExpression = not(lessThanOrEqual(C_BIGINT, nullLiteral())); + originalExpression = not(lessThanOrEqual(A, nullLiteral())); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); assertTrue(result.getTupleDomain().isNone()); - originalExpression = not(equal(C_BIGINT, nullLiteral())); + originalExpression = not(equal(A, nullLiteral())); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); assertTrue(result.getTupleDomain().isNone()); - originalExpression = not(equal(C_COLOR, nullLiteral())); + originalExpression = not(equal(J, nullLiteral())); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); assertTrue(result.getTupleDomain().isNone()); - originalExpression = not(notEqual(C_BIGINT, nullLiteral())); + originalExpression = not(notEqual(A, nullLiteral())); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); assertTrue(result.getTupleDomain().isNone()); - originalExpression = not(notEqual(C_COLOR, nullLiteral())); + originalExpression = not(notEqual(J, nullLiteral())); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); assertTrue(result.getTupleDomain().isNone()); - originalExpression = not(isDistinctFrom(C_BIGINT, nullLiteral())); + originalExpression = not(isDistinctFrom(A, nullLiteral())); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.onlyNull(BIGINT)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.onlyNull(BIGINT)))); - originalExpression = not(isDistinctFrom(C_COLOR, nullLiteral())); + originalExpression = not(isDistinctFrom(J, nullLiteral())); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_COLOR, Domain.onlyNull(COLOR)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(J, Domain.onlyNull(COLOR)))); } @Test @@ -779,84 +764,84 @@ public void testFromComparisonsWithImplictCoercions() throws Exception { // B is a double column. Check that it can be compared against longs - Expression originalExpression = greaterThan(C_DOUBLE, bigintLiteral(2L)); + Expression originalExpression = greaterThan(B, bigintLiteral(2L)); ExtractionResult result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_DOUBLE, Domain.create(ValueSet.ofRanges(Range.greaterThan(DOUBLE, 2.0)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(B, Domain.create(ValueSet.ofRanges(Range.greaterThan(DOUBLE, 2.0)), false)))); // C is a string column. Check that it can be compared. - originalExpression = greaterThan(C_VARCHAR, stringLiteral("test")); + originalExpression = greaterThan(C, stringLiteral("test")); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_VARCHAR, Domain.create(ValueSet.ofRanges(Range.greaterThan(VARCHAR, utf8Slice("test"))), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C, Domain.create(ValueSet.ofRanges(Range.greaterThan(VARCHAR, utf8Slice("test"))), false)))); // A is a long column. Check that it can be compared against doubles - originalExpression = greaterThan(C_BIGINT, doubleLiteral(2.0)); + originalExpression = greaterThan(A, doubleLiteral(2.0)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 2L)), false)))); - originalExpression = greaterThan(C_BIGINT, doubleLiteral(2.1)); + originalExpression = greaterThan(A, doubleLiteral(2.1)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 2L)), false)))); - originalExpression = greaterThanOrEqual(C_BIGINT, doubleLiteral(2.0)); + originalExpression = greaterThanOrEqual(A, doubleLiteral(2.0)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.greaterThanOrEqual(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.greaterThanOrEqual(BIGINT, 2L)), false)))); - originalExpression = greaterThanOrEqual(C_BIGINT, doubleLiteral(2.1)); + originalExpression = greaterThanOrEqual(A, doubleLiteral(2.1)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 2L)), false)))); - originalExpression = lessThan(C_BIGINT, doubleLiteral(2.0)); + originalExpression = lessThan(A, doubleLiteral(2.0)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L)), false)))); - originalExpression = lessThan(C_BIGINT, doubleLiteral(2.1)); + originalExpression = lessThan(A, doubleLiteral(2.1)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(BIGINT, 2L)), false)))); - originalExpression = lessThanOrEqual(C_BIGINT, doubleLiteral(2.0)); + originalExpression = lessThanOrEqual(A, doubleLiteral(2.0)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(BIGINT, 2L)), false)))); - originalExpression = lessThanOrEqual(C_BIGINT, doubleLiteral(2.1)); + originalExpression = lessThanOrEqual(A, doubleLiteral(2.1)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(BIGINT, 2L)), false)))); - originalExpression = equal(C_BIGINT, doubleLiteral(2.0)); + originalExpression = equal(A, doubleLiteral(2.0)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 2L)), false)))); - originalExpression = equal(C_BIGINT, doubleLiteral(2.1)); + originalExpression = equal(A, doubleLiteral(2.1)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.none(BIGINT)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.none(BIGINT)))); - originalExpression = notEqual(C_BIGINT, doubleLiteral(2.0)); + originalExpression = notEqual(A, doubleLiteral(2.0)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L), Range.greaterThan(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L), Range.greaterThan(BIGINT, 2L)), false)))); - originalExpression = notEqual(C_BIGINT, doubleLiteral(2.1)); + originalExpression = notEqual(A, doubleLiteral(2.1)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.notNull(BIGINT)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.notNull(BIGINT)))); - originalExpression = isDistinctFrom(C_BIGINT, doubleLiteral(2.0)); + originalExpression = isDistinctFrom(A, doubleLiteral(2.0)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L), Range.greaterThan(BIGINT, 2L)), true)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L), Range.greaterThan(BIGINT, 2L)), true)))); - originalExpression = isDistinctFrom(C_BIGINT, doubleLiteral(2.1)); + originalExpression = isDistinctFrom(A, doubleLiteral(2.1)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); assertTrue(result.getTupleDomain().isAll()); @@ -864,84 +849,84 @@ public void testFromComparisonsWithImplictCoercions() // Test complements // B is a double column. Check that it can be compared against longs - originalExpression = not(greaterThan(C_DOUBLE, bigintLiteral(2L))); + originalExpression = not(greaterThan(B, bigintLiteral(2L))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_DOUBLE, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(DOUBLE, 2.0)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(B, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(DOUBLE, 2.0)), false)))); // C is a string column. Check that it can be compared. - originalExpression = not(greaterThan(C_VARCHAR, stringLiteral("test"))); + originalExpression = not(greaterThan(C, stringLiteral("test"))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_VARCHAR, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(VARCHAR, utf8Slice("test"))), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(VARCHAR, utf8Slice("test"))), false)))); // A is a long column. Check that it can be compared against doubles - originalExpression = not(greaterThan(C_BIGINT, doubleLiteral(2.0))); + originalExpression = not(greaterThan(A, doubleLiteral(2.0))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(BIGINT, 2L)), false)))); - originalExpression = not(greaterThan(C_BIGINT, doubleLiteral(2.1))); + originalExpression = not(greaterThan(A, doubleLiteral(2.1))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(BIGINT, 2L)), false)))); - originalExpression = not(greaterThanOrEqual(C_BIGINT, doubleLiteral(2.0))); + originalExpression = not(greaterThanOrEqual(A, doubleLiteral(2.0))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L)), false)))); - originalExpression = not(greaterThanOrEqual(C_BIGINT, doubleLiteral(2.1))); + originalExpression = not(greaterThanOrEqual(A, doubleLiteral(2.1))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThanOrEqual(BIGINT, 2L)), false)))); - originalExpression = not(lessThan(C_BIGINT, doubleLiteral(2.0))); + originalExpression = not(lessThan(A, doubleLiteral(2.0))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.greaterThanOrEqual(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.greaterThanOrEqual(BIGINT, 2L)), false)))); - originalExpression = not(lessThan(C_BIGINT, doubleLiteral(2.1))); + originalExpression = not(lessThan(A, doubleLiteral(2.1))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 2L)), false)))); - originalExpression = not(lessThanOrEqual(C_BIGINT, doubleLiteral(2.0))); + originalExpression = not(lessThanOrEqual(A, doubleLiteral(2.0))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 2L)), false)))); - originalExpression = not(lessThanOrEqual(C_BIGINT, doubleLiteral(2.1))); + originalExpression = not(lessThanOrEqual(A, doubleLiteral(2.1))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.greaterThan(BIGINT, 2L)), false)))); - originalExpression = not(equal(C_BIGINT, doubleLiteral(2.0))); + originalExpression = not(equal(A, doubleLiteral(2.0))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L), Range.greaterThan(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L), Range.greaterThan(BIGINT, 2L)), false)))); - originalExpression = not(equal(C_BIGINT, doubleLiteral(2.1))); + originalExpression = not(equal(A, doubleLiteral(2.1))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.notNull(BIGINT)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.notNull(BIGINT)))); - originalExpression = not(notEqual(C_BIGINT, doubleLiteral(2.0))); + originalExpression = not(notEqual(A, doubleLiteral(2.0))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 2L)), false)))); - originalExpression = not(notEqual(C_BIGINT, doubleLiteral(2.1))); + originalExpression = not(notEqual(A, doubleLiteral(2.1))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.none(BIGINT)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.none(BIGINT)))); - originalExpression = not(isDistinctFrom(C_BIGINT, doubleLiteral(2.0))); + originalExpression = not(isDistinctFrom(A, doubleLiteral(2.0))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 2L)), false)))); - originalExpression = not(isDistinctFrom(C_BIGINT, doubleLiteral(2.1))); + originalExpression = not(isDistinctFrom(A, doubleLiteral(2.1))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); assertTrue(result.getTupleDomain().isNone()); @@ -951,25 +936,25 @@ public void testFromComparisonsWithImplictCoercions() public void testFromUnprocessableInPredicate() throws Exception { - Expression originalExpression = new InPredicate(unprocessableExpression1(C_BIGINT), new InListExpression(ImmutableList.of(TRUE_LITERAL))); + Expression originalExpression = new InPredicate(unprocessableExpression1(A), new InListExpression(ImmutableList.of(TRUE_LITERAL))); ExtractionResult result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), originalExpression); assertTrue(result.getTupleDomain().isAll()); - originalExpression = new InPredicate(C_BOOLEAN.toSymbolReference(), new InListExpression(ImmutableList.of(unprocessableExpression1(C_BOOLEAN)))); + originalExpression = new InPredicate(D.toSymbolReference(), new InListExpression(ImmutableList.of(unprocessableExpression1(D)))); result = fromPredicate(originalExpression); - assertEquals(result.getRemainingExpression(), equal(C_BOOLEAN, unprocessableExpression1(C_BOOLEAN))); + assertEquals(result.getRemainingExpression(), equal(D, unprocessableExpression1(D))); assertTrue(result.getTupleDomain().isAll()); - originalExpression = new InPredicate(C_BOOLEAN.toSymbolReference(), new InListExpression(ImmutableList.of(TRUE_LITERAL, unprocessableExpression1(C_BOOLEAN)))); + originalExpression = new InPredicate(D.toSymbolReference(), new InListExpression(ImmutableList.of(TRUE_LITERAL, unprocessableExpression1(D)))); result = fromPredicate(originalExpression); - assertEquals(result.getRemainingExpression(), or(equal(C_BOOLEAN, TRUE_LITERAL), equal(C_BOOLEAN, unprocessableExpression1(C_BOOLEAN)))); + assertEquals(result.getRemainingExpression(), or(equal(D, TRUE_LITERAL), equal(D, unprocessableExpression1(D)))); assertTrue(result.getTupleDomain().isAll()); // Test complement - originalExpression = not(new InPredicate(C_BOOLEAN.toSymbolReference(), new InListExpression(ImmutableList.of(unprocessableExpression1(C_BOOLEAN))))); + originalExpression = not(new InPredicate(D.toSymbolReference(), new InListExpression(ImmutableList.of(unprocessableExpression1(D))))); result = fromPredicate(originalExpression); - assertEquals(result.getRemainingExpression(), not(equal(C_BOOLEAN, unprocessableExpression1(C_BOOLEAN)))); + assertEquals(result.getRemainingExpression(), not(equal(D, unprocessableExpression1(D)))); assertTrue(result.getTupleDomain().isAll()); } @@ -977,35 +962,35 @@ public void testFromUnprocessableInPredicate() public void testFromInPredicate() throws Exception { - Expression originalExpression = in(C_BIGINT, ImmutableList.of(1L)); + Expression originalExpression = in(A, ImmutableList.of(1L)); ExtractionResult result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.singleValue(BIGINT, 1L)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.singleValue(BIGINT, 1L)))); - originalExpression = in(C_COLOR, ImmutableList.of(colorLiteral(COLOR_VALUE_1))); + originalExpression = in(J, ImmutableList.of(colorLiteral(COLOR_VALUE_1))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_COLOR, Domain.singleValue(COLOR, COLOR_VALUE_1)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(J, Domain.singleValue(COLOR, COLOR_VALUE_1)))); - originalExpression = in(C_BIGINT, ImmutableList.of(1L, 2L)); + originalExpression = in(A, ImmutableList.of(1L, 2L)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 1L), Range.equal(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.equal(BIGINT, 1L), Range.equal(BIGINT, 2L)), false)))); - originalExpression = in(C_COLOR, ImmutableList.of(colorLiteral(COLOR_VALUE_1), colorLiteral(COLOR_VALUE_2))); + originalExpression = in(J, ImmutableList.of(colorLiteral(COLOR_VALUE_1), colorLiteral(COLOR_VALUE_2))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_COLOR, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1, COLOR_VALUE_2), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(J, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1, COLOR_VALUE_2), false)))); - originalExpression = not(in(C_BIGINT, ImmutableList.of(1L, 2L))); + originalExpression = not(in(A, ImmutableList.of(1L, 2L))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 1L), Range.range(BIGINT, 1L, false, 2L, false), Range.greaterThan(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 1L), Range.range(BIGINT, 1L, false, 2L, false), Range.greaterThan(BIGINT, 2L)), false)))); - originalExpression = not(in(C_COLOR, ImmutableList.of(colorLiteral(COLOR_VALUE_1), colorLiteral(COLOR_VALUE_2)))); + originalExpression = not(in(J, ImmutableList.of(colorLiteral(COLOR_VALUE_1), colorLiteral(COLOR_VALUE_2)))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_COLOR, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1, COLOR_VALUE_2).complement(), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(J, Domain.create(ValueSet.of(COLOR, COLOR_VALUE_1, COLOR_VALUE_2).complement(), false)))); // TODO update domain translator to properly handle cast // originalExpression = in(A, Arrays.asList(1L, 2L, (Expression) null)); @@ -1033,86 +1018,86 @@ public void testFromInPredicate() public void testFromBetweenPredicate() throws Exception { - Expression originalExpression = between(C_BIGINT, bigintLiteral(1L), bigintLiteral(2L)); + Expression originalExpression = between(A, bigintLiteral(1L), bigintLiteral(2L)); ExtractionResult result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.range(BIGINT, 1L, true, 2L, true)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.range(BIGINT, 1L, true, 2L, true)), false)))); - originalExpression = between(C_BIGINT, bigintLiteral(1L), doubleLiteral(2.1)); + originalExpression = between(A, bigintLiteral(1L), doubleLiteral(2.1)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.range(BIGINT, 1L, true, 2L, true)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.range(BIGINT, 1L, true, 2L, true)), false)))); - originalExpression = between(C_BIGINT, bigintLiteral(1L), nullLiteral()); + originalExpression = between(A, bigintLiteral(1L), nullLiteral()); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); assertTrue(result.getTupleDomain().isNone()); // Test complements - originalExpression = not(between(C_BIGINT, bigintLiteral(1L), bigintLiteral(2L))); + originalExpression = not(between(A, bigintLiteral(1L), bigintLiteral(2L))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 1L), Range.greaterThan(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 1L), Range.greaterThan(BIGINT, 2L)), false)))); - originalExpression = not(between(C_BIGINT, bigintLiteral(1L), doubleLiteral(2.1))); + originalExpression = not(between(A, bigintLiteral(1L), doubleLiteral(2.1))); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 1L), Range.greaterThan(BIGINT, 2L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 1L), Range.greaterThan(BIGINT, 2L)), false)))); - originalExpression = not(between(C_BIGINT, bigintLiteral(1L), nullLiteral())); + originalExpression = not(between(A, bigintLiteral(1L), nullLiteral())); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 1L)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 1L)), false)))); } @Test public void testFromIsNullPredicate() throws Exception { - Expression originalExpression = isNull(C_BIGINT); + Expression originalExpression = isNull(A); ExtractionResult result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.onlyNull(BIGINT)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.onlyNull(BIGINT)))); - originalExpression = isNull(C_HYPER_LOG_LOG); + originalExpression = isNull(K); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_HYPER_LOG_LOG, Domain.onlyNull(HYPER_LOG_LOG)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(K, Domain.onlyNull(HYPER_LOG_LOG)))); - originalExpression = not(isNull(C_BIGINT)); + originalExpression = not(isNull(A)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.notNull(BIGINT)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.notNull(BIGINT)))); - originalExpression = not(isNull(C_HYPER_LOG_LOG)); + originalExpression = not(isNull(K)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_HYPER_LOG_LOG, Domain.notNull(HYPER_LOG_LOG)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(K, Domain.notNull(HYPER_LOG_LOG)))); } @Test public void testFromIsNotNullPredicate() throws Exception { - Expression originalExpression = isNotNull(C_BIGINT); + Expression originalExpression = isNotNull(A); ExtractionResult result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.notNull(BIGINT)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.notNull(BIGINT)))); - originalExpression = isNotNull(C_HYPER_LOG_LOG); + originalExpression = isNotNull(K); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_HYPER_LOG_LOG, Domain.notNull(HYPER_LOG_LOG)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(K, Domain.notNull(HYPER_LOG_LOG)))); - originalExpression = not(isNotNull(C_BIGINT)); + originalExpression = not(isNotNull(A)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_BIGINT, Domain.onlyNull(BIGINT)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(A, Domain.onlyNull(BIGINT)))); - originalExpression = not(isNotNull(C_HYPER_LOG_LOG)); + originalExpression = not(isNotNull(K)); result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_HYPER_LOG_LOG, Domain.onlyNull(HYPER_LOG_LOG)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(K, Domain.onlyNull(HYPER_LOG_LOG)))); } @Test @@ -1159,127 +1144,184 @@ public void testFromNullLiteralPredicate() public void testExpressionConstantFolding() throws Exception { - Expression originalExpression = comparison(GREATER_THAN, C_VARBINARY.toSymbolReference(), function("from_hex", stringLiteral("123456"))); + Expression originalExpression = comparison(GREATER_THAN, L.toSymbolReference(), function("from_hex", stringLiteral("123456"))); ExtractionResult result = fromPredicate(originalExpression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); Slice value = Slices.wrappedBuffer(BaseEncoding.base16().decode("123456")); - assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(C_VARBINARY, Domain.create(ValueSet.ofRanges(Range.greaterThan(VARBINARY, value)), false)))); + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(L, Domain.create(ValueSet.ofRanges(Range.greaterThan(VARBINARY, value)), false)))); Expression expression = toPredicate(result.getTupleDomain()); - assertEquals(expression, comparison(GREATER_THAN, C_VARBINARY.toSymbolReference(), varbinaryLiteral(value))); + assertEquals(expression, comparison(GREATER_THAN, L.toSymbolReference(), varbinaryLiteral(value))); } @Test - public void testNumericTypeTranslation() + public void testBigintComparedToDoubleExpression() throws Exception { - List translationChain = ImmutableList.of( - new NumericValues<>(C_DOUBLE, -1.0 * Double.MAX_VALUE, -22.0, -44.5556836, 23.0, 44.5556789, Double.MAX_VALUE), - new NumericValues<>(C_REAL, realValue(-1.0f * Float.MAX_VALUE), realValue(-22.0f), realValue(-44.555687f), realValue(23.0f), realValue(44.555676f), realValue(Float.MAX_VALUE)), - new NumericValues<>(C_DECIMAL_26_5, longDecimal("-999999999999999999999.99999"), longDecimal("-22.00000"), longDecimal("-44.55569"), longDecimal("23.00000"), longDecimal("44.55567"), longDecimal("999999999999999999999.99999")), - new NumericValues<>(C_DECIMAL_23_4, longDecimal("-9999999999999999999.9999"), longDecimal("-22.0000"), longDecimal("-44.5557"), longDecimal("23.0000"), longDecimal("44.5556"), longDecimal("9999999999999999999.9999")), - new NumericValues<>(C_BIGINT, Long.MIN_VALUE, -22L, -45L, 23L, 44L, Long.MAX_VALUE), - new NumericValues<>(C_DECIMAL_21_3, longDecimal("-999999999999999999.999"), longDecimal("-22.000"), longDecimal("-44.556"), longDecimal("23.000"), longDecimal("44.555"), longDecimal("999999999999999999.999")), - new NumericValues<>(C_DECIMAL_12_2, shortDecimal("-9999999999.99"), shortDecimal("-22.00"), shortDecimal("-44.56"), shortDecimal("23.00"), shortDecimal("44.55"), shortDecimal("9999999999.99")), - new NumericValues<>(C_INTEGER, (long) Integer.MIN_VALUE, -22L, -45L, 23L, 44L, (long) Integer.MAX_VALUE), - new NumericValues<>(C_DECIMAL_6_1, shortDecimal("-99999.9"), shortDecimal("-22.0"), shortDecimal("-44.6"), shortDecimal("23.0"), shortDecimal("44.5"), shortDecimal("99999.9")), - new NumericValues<>(C_SMALLINT, (long) Short.MIN_VALUE, -22L, -45L, 23L, 44L, (long) Short.MAX_VALUE), - new NumericValues<>(C_DECIMAL_3_0, shortDecimal("-999"), shortDecimal("-22"), shortDecimal("-45"), shortDecimal("23"), shortDecimal("44"), shortDecimal("999")), - new NumericValues<>(C_TINYINT, (long) Byte.MIN_VALUE, -22L, -45L, 23L, 44L, (long) Byte.MAX_VALUE), - new NumericValues<>(C_DECIMAL_2_0, shortDecimal("-99"), shortDecimal("-22"), shortDecimal("-45"), shortDecimal("23"), shortDecimal("44"), shortDecimal("99")) - ); - - for (int literalIndex = 0; literalIndex < translationChain.size(); literalIndex++) { - for (int columnIndex = literalIndex + 1; columnIndex < translationChain.size(); columnIndex++) { - NumericValues literal = translationChain.get(literalIndex); - NumericValues column = translationChain.get(columnIndex); - testNumericTypeTranslation(column, literal); - } - } + // greater than or equal + testSimpleComparison(greaterThanOrEqual(A, doubleLiteral(2.5)), A, Range.greaterThan(BIGINT, 2L)); + testSimpleComparison(greaterThanOrEqual(A, doubleLiteral(2.0)), A, Range.greaterThanOrEqual(BIGINT, 2L)); + testSimpleComparison(greaterThanOrEqual(A, doubleLiteral(-2.5)), A, Range.greaterThan(BIGINT, -3L)); + testSimpleComparison(greaterThanOrEqual(A, doubleLiteral(-2.0)), A, Range.greaterThanOrEqual(BIGINT, -2L)); + testSimpleComparison(greaterThanOrEqual(A, doubleLiteral(0x1p64)), A, Range.greaterThan(BIGINT, Long.MAX_VALUE)); + testSimpleComparison(greaterThanOrEqual(A, doubleLiteral(-0x1p64)), A, Range.greaterThanOrEqual(BIGINT, Long.MIN_VALUE)); + + // greater than + testSimpleComparison(greaterThan(A, doubleLiteral(2.5)), A, Range.greaterThan(BIGINT, 2L)); + testSimpleComparison(greaterThan(A, doubleLiteral(2.0)), A, Range.greaterThan(BIGINT, 2L)); + testSimpleComparison(greaterThan(A, doubleLiteral(-2.5)), A, Range.greaterThan(BIGINT, -3L)); + testSimpleComparison(greaterThan(A, doubleLiteral(-2.0)), A, Range.greaterThan(BIGINT, -2L)); + testSimpleComparison(greaterThan(A, doubleLiteral(0x1p64)), A, Range.greaterThan(BIGINT, Long.MAX_VALUE)); + testSimpleComparison(greaterThan(A, doubleLiteral(-0x1p64)), A, Range.greaterThanOrEqual(BIGINT, Long.MIN_VALUE)); + + // less than or equal + testSimpleComparison(lessThanOrEqual(A, doubleLiteral(2.5)), A, Range.lessThanOrEqual(BIGINT, 2L)); + testSimpleComparison(lessThanOrEqual(A, doubleLiteral(2.0)), A, Range.lessThanOrEqual(BIGINT, 2L)); + testSimpleComparison(lessThanOrEqual(A, doubleLiteral(-2.5)), A, Range.lessThanOrEqual(BIGINT, -3L)); + testSimpleComparison(lessThanOrEqual(A, doubleLiteral(-2.0)), A, Range.lessThanOrEqual(BIGINT, -2L)); + testSimpleComparison(lessThanOrEqual(A, doubleLiteral(0x1p64)), A, Range.lessThanOrEqual(BIGINT, Long.MAX_VALUE)); + testSimpleComparison(lessThanOrEqual(A, doubleLiteral(-0x1p64)), A, Range.lessThan(BIGINT, Long.MIN_VALUE)); + + // less than + testSimpleComparison(lessThan(A, doubleLiteral(2.5)), A, Range.lessThanOrEqual(BIGINT, 2L)); + testSimpleComparison(lessThan(A, doubleLiteral(2.0)), A, Range.lessThan(BIGINT, 2L)); + testSimpleComparison(lessThan(A, doubleLiteral(-2.5)), A, Range.lessThanOrEqual(BIGINT, -3L)); + testSimpleComparison(lessThan(A, doubleLiteral(-2.0)), A, Range.lessThan(BIGINT, -2L)); + testSimpleComparison(lessThan(A, doubleLiteral(0x1p64)), A, Range.lessThanOrEqual(BIGINT, Long.MAX_VALUE)); + testSimpleComparison(lessThan(A, doubleLiteral(-0x1p64)), A, Range.lessThan(BIGINT, Long.MIN_VALUE)); + + // equal + testSimpleComparison(equal(A, doubleLiteral(2.5)), A, Domain.none(BIGINT)); + testSimpleComparison(equal(A, doubleLiteral(2.0)), A, Range.equal(BIGINT, 2L)); + testSimpleComparison(equal(A, doubleLiteral(-2.5)), A, Domain.none(BIGINT)); + testSimpleComparison(equal(A, doubleLiteral(-2.0)), A, Range.equal(BIGINT, -2L)); + testSimpleComparison(equal(A, doubleLiteral(0x1p64)), A, Domain.none(BIGINT)); + testSimpleComparison(equal(A, doubleLiteral(-0x1p64)), A, Domain.none(BIGINT)); + + // not equal + testSimpleComparison(notEqual(A, doubleLiteral(2.5)), A, Domain.notNull(BIGINT)); + testSimpleComparison(notEqual(A, doubleLiteral(2.0)), A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L), Range.greaterThan(BIGINT, 2L)), false)); + testSimpleComparison(notEqual(A, doubleLiteral(-2.5)), A, Domain.notNull(BIGINT)); + testSimpleComparison(notEqual(A, doubleLiteral(-2.0)), A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, -2L), Range.greaterThan(BIGINT, -2L)), false)); + testSimpleComparison(notEqual(A, doubleLiteral(0x1p64)), A, Domain.notNull(BIGINT)); + testSimpleComparison(notEqual(A, doubleLiteral(-0x1p64)), A, Domain.notNull(BIGINT)); + + // is distinct from + testSimpleComparison(isDistinctFrom(A, doubleLiteral(2.5)), A, Domain.all(BIGINT)); + testSimpleComparison(isDistinctFrom(A, doubleLiteral(2.0)), A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, 2L), Range.greaterThan(BIGINT, 2L)), true)); + testSimpleComparison(isDistinctFrom(A, doubleLiteral(-2.5)), A, Domain.all(BIGINT)); + testSimpleComparison(isDistinctFrom(A, doubleLiteral(-2.0)), A, Domain.create(ValueSet.ofRanges(Range.lessThan(BIGINT, -2L), Range.greaterThan(BIGINT, -2L)), true)); + testSimpleComparison(isDistinctFrom(A, doubleLiteral(0x1p64)), A, Domain.all(BIGINT)); + testSimpleComparison(isDistinctFrom(A, doubleLiteral(-0x1p64)), A, Domain.all(BIGINT)); } - private void testNumericTypeTranslation(NumericValues columnValues, NumericValues literalValues) + @Test + public void testIntegerComparedToDoubleExpression() + throws Exception { - Symbol columnSymbol = columnValues.getColumn(); - Type columnType = columnValues.getType(); - Type literalType = literalValues.getType(); + // greater than or equal + testSimpleComparison(greaterThanOrEqual(O, doubleLiteral(2.5)), O, Range.greaterThan(INTEGER, 2L)); + testSimpleComparison(greaterThanOrEqual(O, doubleLiteral(2.0)), O, Range.greaterThanOrEqual(INTEGER, 2L)); + testSimpleComparison(greaterThanOrEqual(O, doubleLiteral(0x1p32)), O, Range.greaterThan(INTEGER, (long) Integer.MAX_VALUE)); - Expression max = toExpression(literalValues.getMax(), literalType); - Expression min = toExpression(literalValues.getMin(), literalType); - Expression integerPositive = toExpression(literalValues.getIntegerPositive(), literalType); - Expression integerNegative = toExpression(literalValues.getIntegerNegative(), literalType); - Expression fractionalPositive = toExpression(literalValues.getFractionalPositive(), literalType); - Expression fractionalNegative = toExpression(literalValues.getFractionalNegative(), literalType); + // greater than + testSimpleComparison(greaterThan(O, doubleLiteral(2.5)), O, Range.greaterThan(INTEGER, 2L)); + testSimpleComparison(greaterThan(O, doubleLiteral(2.0)), O, Range.greaterThan(INTEGER, 2L)); + testSimpleComparison(greaterThan(O, doubleLiteral(0x1p32)), O, Range.greaterThan(INTEGER, (long) Integer.MAX_VALUE)); + + // less than or equal + testSimpleComparison(lessThanOrEqual(O, doubleLiteral(-2.5)), O, Range.lessThanOrEqual(INTEGER, -3L)); + testSimpleComparison(lessThanOrEqual(O, doubleLiteral(-2.0)), O, Range.lessThanOrEqual(INTEGER, -2L)); + testSimpleComparison(lessThanOrEqual(O, doubleLiteral(-0x1p32)), O, Range.lessThan(INTEGER, (long) Integer.MIN_VALUE)); + + // less than + testSimpleComparison(lessThan(O, doubleLiteral(-2.5)), O, Range.lessThanOrEqual(INTEGER, -3L)); + testSimpleComparison(lessThan(O, doubleLiteral(-2.0)), O, Range.lessThan(INTEGER, -2L)); + testSimpleComparison(lessThan(O, doubleLiteral(-0x1p32)), O, Range.lessThan(INTEGER, (long) Integer.MIN_VALUE)); + + // equal + testSimpleComparison(equal(O, doubleLiteral(2.5)), O, Domain.none(INTEGER)); + testSimpleComparison(equal(O, doubleLiteral(2.0)), O, Range.equal(INTEGER, 2L)); + + // not equal + testSimpleComparison(notEqual(O, doubleLiteral(2.5)), O, Domain.notNull(INTEGER)); + testSimpleComparison(notEqual(O, doubleLiteral(2.0)), O, Domain.create(ValueSet.ofRanges(Range.lessThan(INTEGER, 2L), Range.greaterThan(INTEGER, 2L)), false)); + + // is distinct from + testSimpleComparison(isDistinctFrom(O, doubleLiteral(2.5)), O, Domain.all(INTEGER)); + testSimpleComparison(isDistinctFrom(O, doubleLiteral(2.0)), O, Domain.create(ValueSet.ofRanges(Range.lessThan(INTEGER, 2L), Range.greaterThan(INTEGER, 2L)), true)); + } + @Test + public void testIntegerComparedToBigintExpression() + throws Exception + { // greater than or equal - testSimpleComparison(greaterThanOrEqual(columnSymbol, integerPositive), columnSymbol, Range.greaterThanOrEqual(columnType, columnValues.getIntegerPositive())); - testSimpleComparison(greaterThanOrEqual(columnSymbol, integerNegative), columnSymbol, Range.greaterThanOrEqual(columnType, columnValues.getIntegerNegative())); - testSimpleComparison(greaterThanOrEqual(columnSymbol, max), columnSymbol, Range.greaterThan(columnType, columnValues.getMax())); - testSimpleComparison(greaterThanOrEqual(columnSymbol, min), columnSymbol, Range.greaterThanOrEqual(columnType, columnValues.getMin())); - if (literalValues.isFractional()) { - testSimpleComparison(greaterThanOrEqual(columnSymbol, fractionalPositive), columnSymbol, Range.greaterThan(columnType, columnValues.getFractionalPositive())); - testSimpleComparison(greaterThanOrEqual(columnSymbol, fractionalNegative), columnSymbol, Range.greaterThan(columnType, columnValues.getFractionalNegative())); - } + testSimpleComparison(greaterThanOrEqual(O, bigintLiteral(2L)), O, Range.greaterThanOrEqual(INTEGER, 2L)); + testSimpleComparison(greaterThanOrEqual(O, bigintLiteral(Integer.MAX_VALUE + 1L)), O, Range.greaterThan(INTEGER, (long) Integer.MAX_VALUE)); // greater than - testSimpleComparison(greaterThan(columnSymbol, integerPositive), columnSymbol, Range.greaterThan(columnType, columnValues.getIntegerPositive())); - testSimpleComparison(greaterThan(columnSymbol, integerNegative), columnSymbol, Range.greaterThan(columnType, columnValues.getIntegerNegative())); - testSimpleComparison(greaterThan(columnSymbol, max), columnSymbol, Range.greaterThan(columnType, columnValues.getMax())); - testSimpleComparison(greaterThan(columnSymbol, min), columnSymbol, Range.greaterThanOrEqual(columnType, columnValues.getMin())); - if (literalValues.isFractional()) { - testSimpleComparison(greaterThan(columnSymbol, fractionalPositive), columnSymbol, Range.greaterThan(columnType, columnValues.getFractionalPositive())); - testSimpleComparison(greaterThan(columnSymbol, fractionalNegative), columnSymbol, Range.greaterThan(columnType, columnValues.getFractionalNegative())); - } + testSimpleComparison(greaterThan(O, bigintLiteral(2L)), O, Range.greaterThan(INTEGER, 2L)); + testSimpleComparison(greaterThan(O, bigintLiteral(Integer.MAX_VALUE + 1L)), O, Range.greaterThan(INTEGER, (long) Integer.MAX_VALUE)); // less than or equal - testSimpleComparison(lessThanOrEqual(columnSymbol, integerPositive), columnSymbol, Range.lessThanOrEqual(columnType, columnValues.getIntegerPositive())); - testSimpleComparison(lessThanOrEqual(columnSymbol, integerNegative), columnSymbol, Range.lessThanOrEqual(columnType, columnValues.getIntegerNegative())); - testSimpleComparison(lessThanOrEqual(columnSymbol, max), columnSymbol, Range.lessThanOrEqual(columnType, columnValues.getMax())); - testSimpleComparison(lessThanOrEqual(columnSymbol, min), columnSymbol, Range.lessThan(columnType, columnValues.getMin())); - if (literalValues.isFractional()) { - testSimpleComparison(lessThanOrEqual(columnSymbol, fractionalPositive), columnSymbol, Range.lessThanOrEqual(columnType, columnValues.getFractionalPositive())); - testSimpleComparison(lessThanOrEqual(columnSymbol, fractionalNegative), columnSymbol, Range.lessThanOrEqual(columnType, columnValues.getFractionalNegative())); - } + testSimpleComparison(lessThanOrEqual(O, bigintLiteral(-2L)), O, Range.lessThanOrEqual(INTEGER, -2L)); + testSimpleComparison(lessThanOrEqual(O, bigintLiteral(Integer.MIN_VALUE - 1L)), O, Range.lessThan(INTEGER, (long) Integer.MIN_VALUE)); // less than - testSimpleComparison(lessThan(columnSymbol, integerPositive), columnSymbol, Range.lessThan(columnType, columnValues.getIntegerPositive())); - testSimpleComparison(lessThan(columnSymbol, integerNegative), columnSymbol, Range.lessThan(columnType, columnValues.getIntegerNegative())); - testSimpleComparison(lessThan(columnSymbol, max), columnSymbol, Range.lessThanOrEqual(columnType, columnValues.getMax())); - testSimpleComparison(lessThan(columnSymbol, min), columnSymbol, Range.lessThan(columnType, columnValues.getMin())); - if (literalValues.isFractional()) { - testSimpleComparison(lessThan(columnSymbol, fractionalPositive), columnSymbol, Range.lessThanOrEqual(columnType, columnValues.getFractionalPositive())); - testSimpleComparison(lessThan(columnSymbol, fractionalNegative), columnSymbol, Range.lessThanOrEqual(columnType, columnValues.getFractionalNegative())); - } + testSimpleComparison(lessThan(O, bigintLiteral(-2L)), O, Range.lessThan(INTEGER, -2L)); + testSimpleComparison(lessThan(O, bigintLiteral(Integer.MIN_VALUE - 1L)), O, Range.lessThan(INTEGER, (long) Integer.MIN_VALUE)); // equal - testSimpleComparison(equal(columnSymbol, integerPositive), columnSymbol, Range.equal(columnType, columnValues.getIntegerPositive())); - testSimpleComparison(equal(columnSymbol, integerNegative), columnSymbol, Range.equal(columnType, columnValues.getIntegerNegative())); - testSimpleComparison(equal(columnSymbol, max), columnSymbol, Domain.none(columnType)); - testSimpleComparison(equal(columnSymbol, min), columnSymbol, Domain.none(columnType)); - if (literalValues.isFractional()) { - testSimpleComparison(equal(columnSymbol, fractionalPositive), columnSymbol, Domain.none(columnType)); - testSimpleComparison(equal(columnSymbol, fractionalNegative), columnSymbol, Domain.none(columnType)); - } + testSimpleComparison(equal(O, bigintLiteral(Integer.MIN_VALUE - 1L)), O, Domain.none(INTEGER)); + testSimpleComparison(equal(O, bigintLiteral(-2L)), O, Range.equal(INTEGER, -2L)); // not equal - testSimpleComparison(notEqual(columnSymbol, integerPositive), columnSymbol, Domain.create(ValueSet.ofRanges(Range.lessThan(columnType, columnValues.getIntegerPositive()), Range.greaterThan(columnType, columnValues.getIntegerPositive())), false)); - testSimpleComparison(notEqual(columnSymbol, integerNegative), columnSymbol, Domain.create(ValueSet.ofRanges(Range.lessThan(columnType, columnValues.getIntegerNegative()), Range.greaterThan(columnType, columnValues.getIntegerNegative())), false)); - testSimpleComparison(notEqual(columnSymbol, max), columnSymbol, Domain.notNull(columnType)); - testSimpleComparison(notEqual(columnSymbol, min), columnSymbol, Domain.notNull(columnType)); - if (literalValues.isFractional()) { - testSimpleComparison(notEqual(columnSymbol, fractionalPositive), columnSymbol, Domain.notNull(columnType)); - testSimpleComparison(notEqual(columnSymbol, fractionalNegative), columnSymbol, Domain.notNull(columnType)); - } + testSimpleComparison(notEqual(O, bigintLiteral(Integer.MAX_VALUE + 1L)), O, Domain.notNull(INTEGER)); + testSimpleComparison(notEqual(O, bigintLiteral(2L)), O, Domain.create(ValueSet.ofRanges(Range.lessThan(INTEGER, 2L), Range.greaterThan(INTEGER, 2L)), false)); // is distinct from - testSimpleComparison(isDistinctFrom(columnSymbol, integerPositive), columnSymbol, Domain.create(ValueSet.ofRanges(Range.lessThan(columnType, columnValues.getIntegerPositive()), Range.greaterThan(columnType, columnValues.getIntegerPositive())), true)); - testSimpleComparison(isDistinctFrom(columnSymbol, integerNegative), columnSymbol, Domain.create(ValueSet.ofRanges(Range.lessThan(columnType, columnValues.getIntegerNegative()), Range.greaterThan(columnType, columnValues.getIntegerNegative())), true)); - testSimpleComparison(isDistinctFrom(columnSymbol, max), columnSymbol, Domain.all(columnType)); - testSimpleComparison(isDistinctFrom(columnSymbol, min), columnSymbol, Domain.all(columnType)); - if (literalValues.isFractional()) { - testSimpleComparison(isDistinctFrom(columnSymbol, fractionalPositive), columnSymbol, Domain.all(columnType)); - testSimpleComparison(isDistinctFrom(columnSymbol, fractionalNegative), columnSymbol, Domain.all(columnType)); - } + testSimpleComparison(isDistinctFrom(O, bigintLiteral(Integer.MAX_VALUE + 1L)), O, Domain.all(INTEGER)); + testSimpleComparison(isDistinctFrom(O, bigintLiteral(2L)), O, Domain.create(ValueSet.ofRanges(Range.lessThan(INTEGER, 2L), Range.greaterThan(INTEGER, 2L)), true)); + } + + @Test + public void testDecimalComparedToWiderDecimal() + throws Exception + { + // greater than or equal + testSimpleComparison(greaterThanOrEqual(N, decimalLiteral("44.555678")), N, Range.greaterThan(createDecimalType(4, 2), shortDecimal("44.55"))); + testSimpleComparison(greaterThanOrEqual(N, decimalLiteral("99.99")), N, Range.greaterThanOrEqual(createDecimalType(4, 2), shortDecimal("99.99"))); + testSimpleComparison(greaterThanOrEqual(N, decimalLiteral("9999.999")), N, Range.greaterThan(createDecimalType(4, 2), shortDecimal("99.99"))); + + // greater than + testSimpleComparison(greaterThan(N, decimalLiteral("44.555678")), N, Range.greaterThan(createDecimalType(4, 2), shortDecimal("44.55"))); + testSimpleComparison(greaterThan(N, decimalLiteral("44.55")), N, Range.greaterThan(createDecimalType(4, 2), shortDecimal("44.55"))); + testSimpleComparison(greaterThan(N, decimalLiteral("9999.999")), N, Range.greaterThan(createDecimalType(4, 2), shortDecimal("99.99"))); + + // less than or equal + testSimpleComparison(lessThanOrEqual(N, decimalLiteral("-44.555678")), N, Range.lessThanOrEqual(createDecimalType(4, 2), shortDecimal("-44.56"))); + testSimpleComparison(lessThanOrEqual(N, decimalLiteral("-99.99")), N, Range.lessThanOrEqual(createDecimalType(4, 2), shortDecimal("-99.99"))); + testSimpleComparison(lessThanOrEqual(N, decimalLiteral("-9999.999")), N, Range.lessThan(createDecimalType(4, 2), shortDecimal("-99.99"))); + + // less than + testSimpleComparison(lessThan(N, decimalLiteral("-44.555678")), N, Range.lessThanOrEqual(createDecimalType(4, 2), shortDecimal("-44.56"))); + testSimpleComparison(lessThan(N, decimalLiteral("-99.99")), N, Range.lessThan(createDecimalType(4, 2), shortDecimal("-99.99"))); + testSimpleComparison(lessThan(N, decimalLiteral("-9999.999")), N, Range.lessThan(createDecimalType(4, 2), shortDecimal("-99.99"))); + + // equal + testSimpleComparison(equal(N, decimalLiteral("-44.555678")), N, Domain.none(createDecimalType(4, 2))); + testSimpleComparison(equal(N, decimalLiteral("99.99")), N, Range.equal(createDecimalType(4, 2), shortDecimal("99.99"))); + + // not equal + testSimpleComparison(notEqual(N, decimalLiteral("-44.555678")), N, Domain.notNull(createDecimalType(4, 2))); + testSimpleComparison(notEqual(N, decimalLiteral("99.99")), N, Domain.create(ValueSet.ofRanges( + Range.lessThan(createDecimalType(4, 2), shortDecimal("99.99")), Range.greaterThan(createDecimalType(4, 2), shortDecimal("99.99"))), false)); + + // is distinct from + testSimpleComparison(isDistinctFrom(N, decimalLiteral("-44.555678")), N, Domain.all(createDecimalType(4, 2))); + testSimpleComparison(isDistinctFrom(N, decimalLiteral("99.99")), N, Domain.create(ValueSet.ofRanges( + Range.lessThan(createDecimalType(4, 2), shortDecimal("99.99")), Range.greaterThan(createDecimalType(4, 2), shortDecimal("99.99"))), true)); } @Test @@ -1287,41 +1329,41 @@ public void testVarcharComparedToCharExpression() throws Exception { // greater than or equal - testSimpleComparison(greaterThanOrEqual(C_CHAR, stringLiteral("123456789")), C_CHAR, Range.greaterThan(createCharType(10), Slices.utf8Slice("12345678"))); - testSimpleComparison(greaterThanOrEqual(C_CHAR, stringLiteral("1234567890")), C_CHAR, Range.greaterThanOrEqual(createCharType(10), Slices.utf8Slice("1234567890"))); - testSimpleComparison(greaterThanOrEqual(C_CHAR, stringLiteral("12345678901")), C_CHAR, Range.greaterThan(createCharType(10), Slices.utf8Slice("1234567890"))); + testSimpleComparison(greaterThanOrEqual(P, stringLiteral("123456789")), P, Range.greaterThan(createCharType(10), Slices.utf8Slice("12345678"))); + testSimpleComparison(greaterThanOrEqual(P, stringLiteral("1234567890")), P, Range.greaterThanOrEqual(createCharType(10), Slices.utf8Slice("1234567890"))); + testSimpleComparison(greaterThanOrEqual(P, stringLiteral("12345678901")), P, Range.greaterThan(createCharType(10), Slices.utf8Slice("1234567890"))); // greater than - testSimpleComparison(greaterThan(C_CHAR, stringLiteral("123456789")), C_CHAR, Range.greaterThan(createCharType(10), Slices.utf8Slice("12345678"))); - testSimpleComparison(greaterThan(C_CHAR, stringLiteral("1234567890")), C_CHAR, Range.greaterThan(createCharType(10), Slices.utf8Slice("1234567890"))); - testSimpleComparison(greaterThan(C_CHAR, stringLiteral("12345678901")), C_CHAR, Range.greaterThan(createCharType(10), Slices.utf8Slice("1234567890"))); + testSimpleComparison(greaterThan(P, stringLiteral("123456789")), P, Range.greaterThan(createCharType(10), Slices.utf8Slice("12345678"))); + testSimpleComparison(greaterThan(P, stringLiteral("1234567890")), P, Range.greaterThan(createCharType(10), Slices.utf8Slice("1234567890"))); + testSimpleComparison(greaterThan(P, stringLiteral("12345678901")), P, Range.greaterThan(createCharType(10), Slices.utf8Slice("1234567890"))); // less than or equal - testSimpleComparison(lessThanOrEqual(C_CHAR, stringLiteral("123456789")), C_CHAR, Range.lessThanOrEqual(createCharType(10), Slices.utf8Slice("12345678"))); - testSimpleComparison(lessThanOrEqual(C_CHAR, stringLiteral("1234567890")), C_CHAR, Range.lessThanOrEqual(createCharType(10), Slices.utf8Slice("1234567890"))); - testSimpleComparison(lessThanOrEqual(C_CHAR, stringLiteral("12345678901")), C_CHAR, Range.lessThanOrEqual(createCharType(10), Slices.utf8Slice("1234567890"))); + testSimpleComparison(lessThanOrEqual(P, stringLiteral("123456789")), P, Range.lessThanOrEqual(createCharType(10), Slices.utf8Slice("12345678"))); + testSimpleComparison(lessThanOrEqual(P, stringLiteral("1234567890")), P, Range.lessThanOrEqual(createCharType(10), Slices.utf8Slice("1234567890"))); + testSimpleComparison(lessThanOrEqual(P, stringLiteral("12345678901")), P, Range.lessThanOrEqual(createCharType(10), Slices.utf8Slice("1234567890"))); // less than - testSimpleComparison(lessThan(C_CHAR, stringLiteral("123456789")), C_CHAR, Range.lessThanOrEqual(createCharType(10), Slices.utf8Slice("12345678"))); - testSimpleComparison(lessThan(C_CHAR, stringLiteral("1234567890")), C_CHAR, Range.lessThan(createCharType(10), Slices.utf8Slice("1234567890"))); - testSimpleComparison(lessThan(C_CHAR, stringLiteral("12345678901")), C_CHAR, Range.lessThanOrEqual(createCharType(10), Slices.utf8Slice("1234567890"))); + testSimpleComparison(lessThan(P, stringLiteral("123456789")), P, Range.lessThanOrEqual(createCharType(10), Slices.utf8Slice("12345678"))); + testSimpleComparison(lessThan(P, stringLiteral("1234567890")), P, Range.lessThan(createCharType(10), Slices.utf8Slice("1234567890"))); + testSimpleComparison(lessThan(P, stringLiteral("12345678901")), P, Range.lessThanOrEqual(createCharType(10), Slices.utf8Slice("1234567890"))); // equal - testSimpleComparison(equal(C_CHAR, stringLiteral("123456789")), C_CHAR, Domain.none(createCharType(10))); - testSimpleComparison(equal(C_CHAR, stringLiteral("1234567890")), C_CHAR, Range.equal(createCharType(10), Slices.utf8Slice("1234567890"))); - testSimpleComparison(equal(C_CHAR, stringLiteral("12345678901")), C_CHAR, Domain.none(createCharType(10))); + testSimpleComparison(equal(P, stringLiteral("123456789")), P, Domain.none(createCharType(10))); + testSimpleComparison(equal(P, stringLiteral("1234567890")), P, Range.equal(createCharType(10), Slices.utf8Slice("1234567890"))); + testSimpleComparison(equal(P, stringLiteral("12345678901")), P, Domain.none(createCharType(10))); // not equal - testSimpleComparison(notEqual(C_CHAR, stringLiteral("123456789")), C_CHAR, Domain.notNull(createCharType(10))); - testSimpleComparison(notEqual(C_CHAR, stringLiteral("1234567890")), C_CHAR, Domain.create(ValueSet.ofRanges( + testSimpleComparison(notEqual(P, stringLiteral("123456789")), P, Domain.notNull(createCharType(10))); + testSimpleComparison(notEqual(P, stringLiteral("1234567890")), P, Domain.create(ValueSet.ofRanges( Range.lessThan(createCharType(10), Slices.utf8Slice("1234567890")), Range.greaterThan(createCharType(10), Slices.utf8Slice("1234567890"))), false)); - testSimpleComparison(notEqual(C_CHAR, stringLiteral("12345678901")), C_CHAR, Domain.notNull(createCharType(10))); + testSimpleComparison(notEqual(P, stringLiteral("12345678901")), P, Domain.notNull(createCharType(10))); // is distinct from - testSimpleComparison(isDistinctFrom(C_CHAR, stringLiteral("123456789")), C_CHAR, Domain.all(createCharType(10))); - testSimpleComparison(isDistinctFrom(C_CHAR, stringLiteral("1234567890")), C_CHAR, Domain.create(ValueSet.ofRanges( + testSimpleComparison(isDistinctFrom(P, stringLiteral("123456789")), P, Domain.all(createCharType(10))); + testSimpleComparison(isDistinctFrom(P, stringLiteral("1234567890")), P, Domain.create(ValueSet.ofRanges( Range.lessThan(createCharType(10), Slices.utf8Slice("1234567890")), Range.greaterThan(createCharType(10), Slices.utf8Slice("1234567890"))), true)); - testSimpleComparison(isDistinctFrom(C_CHAR, stringLiteral("12345678901")), C_CHAR, Domain.all(createCharType(10))); + testSimpleComparison(isDistinctFrom(P, stringLiteral("12345678901")), P, Domain.all(createCharType(10))); } private static ExtractionResult fromPredicate(Expression originalPredicate) @@ -1346,7 +1388,7 @@ private static Expression unprocessableExpression2(Symbol symbol) private static Expression randPredicate(Symbol symbol) { - return comparison(GREATER_THAN, symbol.toSymbolReference(), new FunctionCall(QualifiedName.of("rand"), ImmutableList.of())); + return comparison(GREATER_THAN, symbol.toSymbolReference(), new FunctionCall(QualifiedName.of("rand"), ImmutableList.of())); } private static NotExpression not(Expression expression) @@ -1429,6 +1471,11 @@ private static DoubleLiteral doubleLiteral(double value) return new DoubleLiteral(Double.toString(value)); } + private static DecimalLiteral decimalLiteral(String value) + { + return new DecimalLiteral(value); + } + private static StringLiteral stringLiteral(String value) { return new StringLiteral(value); @@ -1441,12 +1488,12 @@ private static NullLiteral nullLiteral() private static FunctionCall colorLiteral(long value) { - return new FunctionCall(QualifiedName.of(getMagicLiteralFunctionSignature(COLOR).getName()), ImmutableList.of(bigintLiteral(value))); + return new FunctionCall(QualifiedName.of(getMagicLiteralFunctionSignature(COLOR).getName()), ImmutableList.of(bigintLiteral(value))); } private static Expression varbinaryLiteral(Slice value) { - return toExpression(value, VARBINARY); + return LiteralInterpreter.toExpression(value, VARBINARY); } private static FunctionCall function(String functionName, Expression... args) @@ -1459,16 +1506,6 @@ private static Long shortDecimal(String value) return new BigDecimal(value).unscaledValue().longValueExact(); } - private static Slice longDecimal(String value) - { - return encodeScaledValue(new BigDecimal(value)); - } - - private static Long realValue(float value) - { - return (long) Float.floatToIntBits(value); - } - private static void testSimpleComparison(Expression expression, Symbol symbol, Range expectedDomainRange) { testSimpleComparison(expression, symbol, Domain.create(ValueSet.ofRanges(expectedDomainRange), false)); @@ -1478,79 +1515,6 @@ private static void testSimpleComparison(Expression expression, Symbol symbol, D { ExtractionResult result = fromPredicate(expression); assertEquals(result.getRemainingExpression(), TRUE_LITERAL); - TupleDomain actual = result.getTupleDomain(); - TupleDomain expected = withColumnDomains(ImmutableMap.of(symbol, domain)); - if (!actual.equals(expected)) { - fail(format("for comparison [%s] expected %s but found %s", expression.toString(), expected.toString(SESSION), actual.toString(SESSION))); - } - } - - private static class NumericValues - { - private final Symbol column; - private final Type type; - private final T min; - private final T integerNegative; - private final T fractionalNegative; - private final T integerPositive; - private final T fractionalPositive; - private final T max; - - private NumericValues(Symbol column, T min, T integerNegative, T fractionalNegative, T integerPositive, T fractionalPositive, T max) - { - this.column = requireNonNull(column, "column is null"); - this.type = requireNonNull(TYPES.get(column), "type for column not found: " + column); - this.min = requireNonNull(min, "min is null"); - this.integerNegative = requireNonNull(integerNegative, "integerNegative is null"); - this.fractionalNegative = requireNonNull(fractionalNegative, "fractionalNegative is null"); - this.integerPositive = requireNonNull(integerPositive, "integerPositive is null"); - this.fractionalPositive = requireNonNull(fractionalPositive, "fractionalPositive is null"); - this.max = requireNonNull(max, "max is null"); - } - - public Symbol getColumn() - { - return column; - } - - public Type getType() - { - return type; - } - - public T getMin() - { - return min; - } - - public T getIntegerNegative() - { - return integerNegative; - } - - public T getFractionalNegative() - { - return fractionalNegative; - } - - public T getIntegerPositive() - { - return integerPositive; - } - - public T getFractionalPositive() - { - return fractionalPositive; - } - - public T getMax() - { - return max; - } - - public boolean isFractional() - { - return type == DOUBLE || type == REAL || (type instanceof DecimalType && ((DecimalType) type).getScale() > 0); - } + assertEquals(result.getTupleDomain(), withColumnDomains(ImmutableMap.of(symbol, domain))); } } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestEffectivePredicateExtractor.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestEffectivePredicateExtractor.java index 1f1e0ebbeb9d..7fbe1dfc0689 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestEffectivePredicateExtractor.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestEffectivePredicateExtractor.java @@ -151,7 +151,7 @@ public void testAggregation() equals(EE, FE))), ImmutableMap.of(C, fakeFunction("test"), D, fakeFunction("test")), ImmutableMap.of(C, fakeFunctionHandle("test", AGGREGATE), D, fakeFunctionHandle("test", AGGREGATE)), - ImmutableMap.of(), + ImmutableMap.of(), ImmutableList.of(ImmutableList.of(A, B, C)), AggregationNode.Step.FINAL, Optional.empty(), @@ -194,7 +194,7 @@ public void testFilter() { PlanNode node = filter(baseTableScan, and( - greaterThan(AE, new FunctionCall(QualifiedName.of("rand"), ImmutableList.of())), + greaterThan(AE, new FunctionCall(QualifiedName.of("rand"), ImmutableList.of())), lessThan(BE, bigintLiteral(10)))); Expression effectivePredicate = EffectivePredicateExtractor.extract(node, TYPES); @@ -391,7 +391,7 @@ public void testUnion() { ImmutableListMultimap symbolMapping = ImmutableListMultimap.of(A, B, A, C, A, E); PlanNode node = new UnionNode(newId(), - ImmutableList.of( + ImmutableList.of( filter(baseTableScan, greaterThan(AE, bigintLiteral(10))), filter(baseTableScan, and(greaterThan(AE, bigintLiteral(10)), lessThan(AE, bigintLiteral(100)))), filter(baseTableScan, and(greaterThan(AE, bigintLiteral(10)), lessThan(AE, bigintLiteral(100)))) @@ -736,12 +736,12 @@ private static IsNullPredicate isNull(Expression expression) private static FunctionCall fakeFunction(String name) { - return new FunctionCall(QualifiedName.of("test"), ImmutableList.of()); + return new FunctionCall(QualifiedName.of("test"), ImmutableList.of()); } private static Signature fakeFunctionHandle(String name, FunctionKind kind) { - return new Signature(name, kind, TypeSignature.parseTypeSignature(UnknownType.NAME), ImmutableList.of()); + return new Signature(name, kind, TypeSignature.parseTypeSignature(UnknownType.NAME), ImmutableList.of()); } private Set normalizeConjuncts(Expression... conjuncts) @@ -765,11 +765,11 @@ private Set normalizeConjuncts(Expression predicate) Set rewrittenSet = new HashSet<>(); for (Expression expression : EqualityInference.nonInferrableConjuncts(predicate)) { - Expression rewritten = inference.rewriteExpression(expression, Predicates.alwaysTrue()); + Expression rewritten = inference.rewriteExpression(expression, Predicates.alwaysTrue()); Preconditions.checkState(rewritten != null, "Rewrite with full symbol scope should always be possible"); rewrittenSet.add(rewritten); } - rewrittenSet.addAll(inference.generateEqualitiesPartitionedBy(Predicates.alwaysTrue()).getScopeEqualities()); + rewrittenSet.addAll(inference.generateEqualitiesPartitionedBy(Predicates.alwaysTrue()).getScopeEqualities()); return rewrittenSet; } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestEqualityInference.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestEqualityInference.java index 0bde32cdb0ed..8aee0d1e125c 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestEqualityInference.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestEqualityInference.java @@ -190,7 +190,7 @@ public void testEqualityPartitionGeneration() EqualityInference inference = builder.build(); - EqualityInference.EqualityPartition emptyScopePartition = inference.generateEqualitiesPartitionedBy(Predicates.alwaysFalse()); + EqualityInference.EqualityPartition emptyScopePartition = inference.generateEqualitiesPartitionedBy(Predicates.alwaysFalse()); // Cannot generate any scope equalities with no matching symbols assertTrue(emptyScopePartition.getScopeEqualities().isEmpty()); // All equalities should be represented in the inverse scope diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestInterpretedFilterFunction.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestInterpretedFilterFunction.java index 9c5b0237e890..ed65a7d32bc7 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestInterpretedFilterFunction.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestInterpretedFilterFunction.java @@ -15,6 +15,7 @@ import com.facebook.presto.metadata.Metadata; import com.facebook.presto.metadata.MetadataManager; +import com.facebook.presto.spi.type.Type; import com.facebook.presto.sql.parser.SqlParser; import com.facebook.presto.sql.tree.ComparisonExpressionType; import com.facebook.presto.sql.tree.Expression; @@ -192,11 +193,11 @@ public void testComparisonExpressionWithNulls() public static void assertFilter(String expression, boolean expectedValue) { - Expression parsed = createExpression(expression, METADATA, ImmutableMap.of()); + Expression parsed = createExpression(expression, METADATA, ImmutableMap.of()); InterpretedInternalFilterFunction filterFunction = new InterpretedInternalFilterFunction(parsed, - ImmutableMap.of(), - ImmutableMap.of(), + ImmutableMap.of(), + ImmutableMap.of(), METADATA, SQL_PARSER, TEST_SESSION diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestInterpretedProjectionFunction.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestInterpretedProjectionFunction.java index d85d61718608..72e11bbefd06 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestInterpretedProjectionFunction.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestInterpretedProjectionFunction.java @@ -149,17 +149,17 @@ public void testSymbolReference() { Symbol symbol = new Symbol("symbol"); ImmutableMap symbolToInputMappings = ImmutableMap.of(symbol, 0); - assertProjection("symbol", true, symbolToInputMappings, ImmutableMap.of(symbol, BOOLEAN), 0, createBlock(BOOLEAN, true)); - assertProjection("symbol", null, symbolToInputMappings, ImmutableMap.of(symbol, BOOLEAN), 0, createBlock(BOOLEAN, null)); + assertProjection("symbol", true, symbolToInputMappings, ImmutableMap.of(symbol, BOOLEAN), 0, createBlock(BOOLEAN, true)); + assertProjection("symbol", null, symbolToInputMappings, ImmutableMap.of(symbol, BOOLEAN), 0, createBlock(BOOLEAN, null)); - assertProjection("symbol", 42L, symbolToInputMappings, ImmutableMap.of(symbol, BIGINT), 0, createBlock(BIGINT, 42)); - assertProjection("symbol", null, symbolToInputMappings, ImmutableMap.of(symbol, BIGINT), 0, createBlock(BIGINT, null)); + assertProjection("symbol", 42L, symbolToInputMappings, ImmutableMap.of(symbol, BIGINT), 0, createBlock(BIGINT, 42)); + assertProjection("symbol", null, symbolToInputMappings, ImmutableMap.of(symbol, BIGINT), 0, createBlock(BIGINT, null)); - assertProjection("symbol", 11.1, symbolToInputMappings, ImmutableMap.of(symbol, DOUBLE), 0, createBlock(DOUBLE, 11.1)); - assertProjection("symbol", null, symbolToInputMappings, ImmutableMap.of(symbol, DOUBLE), 0, createBlock(DOUBLE, null)); + assertProjection("symbol", 11.1, symbolToInputMappings, ImmutableMap.of(symbol, DOUBLE), 0, createBlock(DOUBLE, 11.1)); + assertProjection("symbol", null, symbolToInputMappings, ImmutableMap.of(symbol, DOUBLE), 0, createBlock(DOUBLE, null)); - assertProjection("symbol", "foo", symbolToInputMappings, ImmutableMap.of(symbol, VARCHAR), 0, createBlock(VARCHAR, "foo")); - assertProjection("symbol", null, symbolToInputMappings, ImmutableMap.of(symbol, VARCHAR), 0, createBlock(VARCHAR, null)); + assertProjection("symbol", "foo", symbolToInputMappings, ImmutableMap.of(symbol, VARCHAR), 0, createBlock(VARCHAR, "foo")); + assertProjection("symbol", null, symbolToInputMappings, ImmutableMap.of(symbol, VARCHAR), 0, createBlock(VARCHAR, null)); } public static void assertProjection(String expression, @Nullable Object expectedValue) @@ -167,8 +167,8 @@ public static void assertProjection(String expression, @Nullable Object expected assertProjection( expression, expectedValue, - ImmutableMap.of(), - ImmutableMap.of(), + ImmutableMap.of(), + ImmutableMap.of(), 0); } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestLogicalPlanner.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestLogicalPlanner.java index 32f74bdf9140..bb62711dcb85 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestLogicalPlanner.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestLogicalPlanner.java @@ -14,7 +14,9 @@ package com.facebook.presto.sql.planner; import com.facebook.presto.spi.predicate.Domain; -import com.facebook.presto.sql.planner.assertions.BasePlanTest; +import com.facebook.presto.sql.planner.assertions.PlanAssert; +import com.facebook.presto.sql.planner.assertions.PlanMatchPattern; +import com.facebook.presto.sql.planner.plan.AggregationNode; import com.facebook.presto.sql.planner.plan.ApplyNode; import com.facebook.presto.sql.planner.plan.EnforceSingleRowNode; import com.facebook.presto.sql.planner.plan.IndexJoinNode; @@ -22,26 +24,23 @@ import com.facebook.presto.sql.planner.plan.PlanNode; import com.facebook.presto.sql.planner.plan.SemiJoinNode; import com.facebook.presto.sql.planner.plan.ValuesNode; +import com.facebook.presto.testing.LocalQueryRunner; +import com.facebook.presto.tpch.TpchConnectorFactory; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.testng.annotations.Test; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.function.Predicate; import static com.facebook.presto.spi.predicate.Domain.singleValue; import static com.facebook.presto.spi.type.VarcharType.createVarcharType; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.aggregation; +import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.aliasPair; import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.any; import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.anyTree; import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.apply; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.constrainedTableScan; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.equiJoinClause; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.expression; import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.filter; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.functionCall; import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.join; import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.node; import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.project; @@ -49,23 +48,39 @@ import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.tableScan; import static com.facebook.presto.sql.planner.plan.JoinNode.Type.INNER; import static com.facebook.presto.sql.planner.plan.JoinNode.Type.LEFT; +import static com.facebook.presto.testing.TestingSession.testSessionBuilder; import static io.airlift.slice.Slices.utf8Slice; import static java.util.Objects.requireNonNull; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.fail; public class TestLogicalPlanner - extends BasePlanTest { + private final LocalQueryRunner queryRunner; + + public TestLogicalPlanner() + { + this.queryRunner = new LocalQueryRunner(testSessionBuilder() + .setCatalog("local") + .setSchema("tiny") + .setSystemProperty("task_concurrency", "1") // these tests don't handle exchanges from local parallel + .build()); + + queryRunner.createCatalog(queryRunner.getDefaultSession().getCatalog().get(), + new TpchConnectorFactory(1), + ImmutableMap.of()); + } + @Test public void testJoin() { assertPlan("SELECT o.orderkey FROM orders o, lineitem l WHERE l.orderkey = o.orderkey", anyTree( - join(INNER, ImmutableList.of(equiJoinClause("ORDERS_OK", "LINEITEM_OK")), + join(INNER, ImmutableList.of(aliasPair("O", "L")), any( - tableScan("orders", ImmutableMap.of("ORDERS_OK", "orderkey"))), + tableScan("orders").withSymbol("orderkey", "O")), anyTree( - tableScan("lineitem", ImmutableMap.of("LINEITEM_OK", "orderkey")))))); + tableScan("lineitem").withSymbol("orderkey", "L"))))); } @Test @@ -73,13 +88,13 @@ public void testUncorrelatedSubqueries() { assertPlan("SELECT * FROM orders WHERE orderkey = (SELECT orderkey FROM lineitem ORDER BY orderkey LIMIT 1)", anyTree( - join(INNER, ImmutableList.of(equiJoinClause("X", "Y")), + join(INNER, ImmutableList.of(aliasPair("X", "Y")), project( - tableScan("orders", ImmutableMap.of("X", "orderkey"))), + tableScan("orders").withSymbol("orderkey", "X")), project( node(EnforceSingleRowNode.class, anyTree( - tableScan("lineitem", ImmutableMap.of("Y", "orderkey")))))))); + tableScan("lineitem").withSymbol("orderkey", "Y"))))))); assertPlan("SELECT * FROM orders WHERE orderkey IN (SELECT orderkey FROM lineitem WHERE linenumber % 4 = 0)", anyTree( @@ -87,9 +102,9 @@ public void testUncorrelatedSubqueries() project( semiJoin("X", "Y", "S", anyTree( - tableScan("orders", ImmutableMap.of("X", "orderkey"))), + tableScan("orders").withSymbol("orderkey", "X")), anyTree( - tableScan("lineitem", ImmutableMap.of("Y", "orderkey")))))))); + tableScan("lineitem").withSymbol("orderkey", "Y"))))))); assertPlan("SELECT * FROM orders WHERE orderkey NOT IN (SELECT orderkey FROM lineitem WHERE linenumber < 0)", anyTree( @@ -97,9 +112,9 @@ public void testUncorrelatedSubqueries() project( semiJoin("X", "Y", "S", anyTree( - tableScan("orders", ImmutableMap.of("X", "orderkey"))), + tableScan("orders").withSymbol("orderkey", "X")), anyTree( - tableScan("lineitem", ImmutableMap.of("Y", "orderkey")))))))); + tableScan("lineitem").withSymbol("orderkey", "Y"))))))); } @Test @@ -113,15 +128,11 @@ public void testPushDownJoinConditionConjunctsToInnerSideBasedOnInheritedPredica "SELECT nationkey FROM nation LEFT OUTER JOIN region " + "ON nation.regionkey = region.regionkey and nation.name = region.name WHERE nation.name = 'blah'", anyTree( - join(LEFT, ImmutableList.of(equiJoinClause("NATION_NAME", "REGION_NAME"), equiJoinClause("NATION_REGIONKEY", "REGION_REGIONKEY")), + join(LEFT, ImmutableList.of(aliasPair("name", "name_1"), aliasPair("regionkey", "regionkey_0")), anyTree( - constrainedTableScan("nation", tableScanConstraint, ImmutableMap.of( - "NATION_NAME", "name", - "NATION_REGIONKEY", "regionkey"))), + tableScan("nation", tableScanConstraint)), anyTree( - constrainedTableScan("region", tableScanConstraint, ImmutableMap.of( - "REGION_NAME", "name", - "REGION_REGIONKEY", "regionkey")))))); + tableScan("region", tableScanConstraint))))); } @Test @@ -138,29 +149,16 @@ public void testSameScalarSubqueryIsAppliedOnlyOnce() countOfMatchingNodes( plan("SELECT * FROM orders o1 JOIN orders o2 ON o1.orderkey = (SELECT 1) AND o2.orderkey = (SELECT 1) AND o1.orderkey + o2.orderkey = (SELECT 1)"), EnforceSingleRowNode.class::isInstance), - 1); - - // one subquery used for "1 IN (SELECT 1)", one subquery used for "2 IN (SELECT 1)" - assertEquals( - countOfMatchingNodes( - plan("SELECT 1 IN (SELECT 1), 2 IN (SELECT 1) WHERE 1 IN (SELECT 1)"), - SemiJoinNode.class::isInstance), 2); } - private static int countOfMatchingNodes(Plan plan, Predicate predicate) + private int countOfMatchingNodes(Plan plan, Predicate predicate) { PlanNodeExtractor planNodeExtractor = new PlanNodeExtractor(predicate); plan.getRoot().accept(planNodeExtractor, null); return planNodeExtractor.getNodes().size(); } - @Test - public void testRemoveUnreferencedScalarInputApplyNodes() - { - assertPlanContainsNoApplyOrJoin("SELECT (SELECT 1)"); - } - @Test public void testSubqueryPruning() { @@ -197,33 +195,25 @@ public void testCorrelatedSubqueries() anyTree( filter("3 = X", apply(ImmutableList.of("X"), - ImmutableMap.of(), - tableScan("orders", ImmutableMap.of("X", "orderkey")), + tableScan("orders").withSymbol("orderkey", "X"), node(EnforceSingleRowNode.class, project( node(ValuesNode.class) )))))); - } - @Test - public void testDoubleNestedCorrelatedSubqueries() - { + // double nesting assertPlan( "SELECT orderkey FROM orders o " + "WHERE 3 IN (SELECT o.custkey FROM lineitem l WHERE (SELECT l.orderkey = o.orderkey))", LogicalPlanner.Stage.OPTIMIZED, anyTree( - filter("OUTER_FILTER", + filter("3 IN (C)", apply(ImmutableList.of("C", "O"), - ImmutableMap.of("OUTER_FILTER", expression("THREE IN (C)")), - project(ImmutableMap.of("THREE", expression("3")), - tableScan("orders", ImmutableMap.of( - "O", "orderkey", - "C", "custkey"))), + project( + tableScan("orders").withSymbol("orderkey", "O").withSymbol("custkey", "C")), anyTree( apply(ImmutableList.of("L"), - ImmutableMap.of(), - tableScan("lineitem", ImmutableMap.of("L", "orderkey")), + tableScan("lineitem").withSymbol("orderkey", "L"), node(EnforceSingleRowNode.class, project( node(ValuesNode.class) @@ -236,17 +226,189 @@ public void testCorrelatedScalarAggregationRewriteToLeftOuterJoin() assertPlan( "SELECT orderkey FROM orders WHERE EXISTS(SELECT 1 WHERE orderkey = 3)", // EXISTS maps to count(*) = 1 anyTree( - filter("FINAL_COUNT > 0", - any( - aggregation(ImmutableMap.of("FINAL_COUNT", functionCall("count", ImmutableList.of("PARTIAL_COUNT"))), - any( - aggregation(ImmutableMap.of("PARTIAL_COUNT", functionCall("count", ImmutableList.of("NON_NULL"))), - any( - join(LEFT, ImmutableList.of(), Optional.of("ORDERKEY = 3"), - any( - tableScan("orders", ImmutableMap.of("ORDERKEY", "orderkey"))), - project(ImmutableMap.of("NON_NULL", expression("true")), - node(ValuesNode.class))))))))))); + filter("count > 0", + anyTree( + node(AggregationNode.class, + anyTree( + join(LEFT, ImmutableList.of(), + anyTree( + tableScan("orders")), + anyTree( + node(ValuesNode.class) + )))))))); + } + + @Test + public void testQuantifiedComparisonEqualsAny() + { + String query = "SELECT orderkey, custkey FROM orders WHERE orderkey = ANY (VALUES ROW(CAST(5 as BIGINT)), ROW(CAST(3 as BIGINT)))"; + assertPlan(query, LogicalPlanner.Stage.CREATED, anyTree( + filter("X IN (Y)", + apply(ImmutableList.of(), + anyTree(tableScan("orders").withSymbol("orderkey", "X")), + anyTree(node(ValuesNode.class).withSymbol("field", "Y"))))) + ); + assertPlan(query, anyTree( + filter("S", + project( + semiJoin("X", "Y", "S", + anyTree(tableScan("orders").withSymbol("orderkey", "X")), + anyTree(node(ValuesNode.class).withSymbol("field", "Y")))))) + ); + } + + @Test + public void testQuantifiedComparisonNotEqualsAll() + { + String query = "SELECT orderkey, custkey FROM orders WHERE orderkey <> ALL (VALUES ROW(CAST(5 as BIGINT)), ROW(CAST(3 as BIGINT)))"; + assertPlan(query, LogicalPlanner.Stage.CREATED, anyTree( + filter("NOT (X IN (Y))", + apply(ImmutableList.of(), + anyTree(tableScan("orders").withSymbol("orderkey", "X")), + anyTree(node(ValuesNode.class).withSymbol("field", "Y"))))) + ); + assertPlan(query, anyTree( + filter("NOT S", + project( + semiJoin("X", "Y", "S", + anyTree(tableScan("orders").withSymbol("orderkey", "X")), + anyTree(node(ValuesNode.class).withSymbol("field", "Y")))))) + ); + } + + @Test + public void testQuantifiedComparisonLessAll() + { + assertOrderedQuantifiedComparison("SELECT orderkey, custkey FROM orders WHERE orderkey < ALL (VALUES CAST(5 as BIGINT), CAST(3 as BIGINT))", + "X < MIN", "X", "min", "MIN"); + } + + @Test + public void testQuantifiedComparisonGreaterEqualAll() + { + assertOrderedQuantifiedComparison("SELECT orderkey, custkey FROM orders WHERE orderkey >= ALL (VALUES CAST(5 as BIGINT), CAST(3 as BIGINT))", + "X >= MAX", "X", "max", "MAX"); + } + + @Test + public void testQuantifiedComparisonLessSome() + { + assertOrderedQuantifiedComparison("SELECT orderkey, custkey FROM orders WHERE orderkey < SOME (VALUES CAST(5 as BIGINT), CAST(3 as BIGINT))", + "X < MAX", "X", "max", "MAX"); + } + + @Test + public void testQuantifiedComparisonGreaterEqualAny() + { + assertOrderedQuantifiedComparison("SELECT orderkey, custkey FROM orders WHERE orderkey >= ANY (VALUES CAST(5 as BIGINT), CAST(3 as BIGINT))", + "X >= MIN", "X", "min", "MIN"); + } + + @Test + public void testQuantifiedComparisonEqualAll() + { + String query = "SELECT orderkey, custkey FROM orders WHERE orderkey = ALL (VALUES CAST(5 as BIGINT), CAST(3 as BIGINT))"; + + assertPlan(query, LogicalPlanner.Stage.CREATED, + anyTree( + project( + filter("MIN = MAX AND X = MIN", + apply(ImmutableList.of(), + project( + tableScan("orders").withSymbol("orderkey", "X")), + node(EnforceSingleRowNode.class, + node(AggregationNode.class, + anyTree( + node(ValuesNode.class)))).withSymbol("min", "MIN").withSymbol("max", "MAX") + ))))); + assertPlan(query, anyTree( + node(JoinNode.class, + anyTree( + tableScan("orders")), + anyTree( + node(AggregationNode.class, + node(ValuesNode.class))))) + ); + } + + @Test + public void testQuantifiedComparisonNotEqualAny() + { + String query = "SELECT orderkey, custkey FROM orders WHERE orderkey <> SOME (VALUES CAST(5 as BIGINT), CAST(3 as BIGINT))"; + + assertPlan(query, LogicalPlanner.Stage.CREATED, + anyTree( + project( + filter("NOT (MIN = MAX AND X = MIN)", + apply(ImmutableList.of(), + project( + tableScan("orders").withSymbol("orderkey", "X")), + node(EnforceSingleRowNode.class, + node(AggregationNode.class, + anyTree( + node(ValuesNode.class)))).withSymbol("min", "MIN").withSymbol("max", "MAX") + ))))); + assertPlan(query, anyTree( + node(JoinNode.class, + tableScan("orders"), + anyTree( + node(AggregationNode.class, + node(ValuesNode.class))))) + ); + } + + private void assertPlan(String sql, PlanMatchPattern pattern) + { + assertPlan(sql, LogicalPlanner.Stage.OPTIMIZED_AND_VALIDATED, pattern); + } + + private void assertPlan(String sql, LogicalPlanner.Stage stage, PlanMatchPattern pattern) + { + queryRunner.inTransaction(transactionSession -> { + Plan actualPlan = queryRunner.createPlan(transactionSession, sql, stage); + PlanAssert.assertPlan(transactionSession, queryRunner.getMetadata(), actualPlan, pattern); + return null; + }); + } + + private Plan plan(String sql) + { + return plan(sql, LogicalPlanner.Stage.OPTIMIZED_AND_VALIDATED); + } + + private Plan plan(String sql, LogicalPlanner.Stage stage) + { + try { + return queryRunner.inTransaction(transactionSession -> queryRunner.createPlan(transactionSession, sql, stage)); + } + catch (RuntimeException ex) { + fail("Invalid SQL: " + sql, ex); + return null; // make compiler happy + } + } + + private void assertOrderedQuantifiedComparison(String query, String filter, String columnMapping, String function, String functionAlias) + { + assertPlan(query, LogicalPlanner.Stage.CREATED, anyTree( + project( + filter(filter, + apply(ImmutableList.of(), + project( + tableScan("orders").withSymbol("orderkey", columnMapping)), + node(EnforceSingleRowNode.class, + node(AggregationNode.class, + anyTree( + node(ValuesNode.class))).withSymbol(function, functionAlias) + )))))); + assertPlan(query, anyTree( + project( + filter(filter, + join(INNER, ImmutableList.of(), + tableScan("orders").withSymbol("orderkey", columnMapping), + node(EnforceSingleRowNode.class, + node(AggregationNode.class, + node(ValuesNode.class)).withSymbol(function, functionAlias) + )))))); } private static final class PlanNodeExtractor diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestPlanMatchingFramework.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestPlanMatchingFramework.java deleted file mode 100644 index ec510cca7b60..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestPlanMatchingFramework.java +++ /dev/null @@ -1,200 +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.sql.planner; - -import com.facebook.presto.sql.planner.assertions.BasePlanTest; -import com.facebook.presto.sql.planner.plan.OutputNode; -import com.facebook.presto.sql.planner.plan.TableScanNode; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import org.testng.annotations.Test; - -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.aggregation; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.any; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.anyTree; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.columnReference; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.equiJoinClause; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.expression; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.functionCall; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.join; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.node; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.output; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.project; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.tableScan; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.values; -import static com.facebook.presto.sql.planner.plan.JoinNode.Type.INNER; - -public class TestPlanMatchingFramework - extends BasePlanTest -{ - @Test - public void testOutput() - { - assertMinimallyOptimizedPlan("SELECT orderkey FROM lineitem", - node(OutputNode.class, - node(TableScanNode.class).withAlias("ORDERKEY", columnReference("lineitem", "orderkey"))) - .withOutputs(ImmutableList.of("ORDERKEY"))); - } - - @Test - public void testOutputSameColumnMultipleTimes() - { - assertMinimallyOptimizedPlan("SELECT orderkey, orderkey FROM lineitem", - output(ImmutableList.of("ORDERKEY", "ORDERKEY"), - tableScan("lineitem", ImmutableMap.of("ORDERKEY", "orderkey")))); - } - - @Test - public void testOutputSameColumnMultipleTimesWithOtherOutputs() - { - assertMinimallyOptimizedPlan("SELECT extendedprice, orderkey, discount, orderkey, linenumber FROM lineitem", - output(ImmutableList.of("ORDERKEY", "ORDERKEY"), - /* - * This is a project node, but this gives us a convenient way to verify that - * visitProject is correctly handled through an anyTree. - */ - anyTree( - tableScan("lineitem", ImmutableMap.of("ORDERKEY", "orderkey"))))); - } - - @Test - public void testUnreferencedSymbolsDontNeedBinding() - { - assertMinimallyOptimizedPlan("SELECT orderkey, 2 FROM lineitem", - output(ImmutableList.of("ORDERKEY"), - anyTree( - tableScan("lineitem", ImmutableMap.of("ORDERKEY", "orderkey"))))); - } - - @Test - public void testAliasConstantFromProject() - { - assertMinimallyOptimizedPlan("SELECT orderkey, 2 FROM lineitem", - output(ImmutableList.of("ORDERKEY", "TWO"), - project(ImmutableMap.of("TWO", expression("2")), - tableScan("lineitem", ImmutableMap.of("ORDERKEY", "orderkey"))))); - } - - @Test - public void testAliasExpressionFromProject() - { - assertMinimallyOptimizedPlan("SELECT orderkey, 1 + orderkey FROM lineitem", - output(ImmutableList.of("ORDERKEY", "EXPRESSION"), - project(ImmutableMap.of("EXPRESSION", expression("CAST(1 AS bigint) + ORDERKEY")), - tableScan("lineitem", ImmutableMap.of("ORDERKEY", "orderkey"))))); - } - - @Test - public void testIdentityAliasFromProject() - { - assertMinimallyOptimizedPlan("SELECT orderkey, 1 + orderkey FROM lineitem", - output(ImmutableList.of("ORDERKEY", "EXPRESSION"), - project(ImmutableMap.of("ORDERKEY", expression("ORDERKEY"), "EXPRESSION", expression("CAST(1 AS bigint) + ORDERKEY")), - tableScan("lineitem", ImmutableMap.of("ORDERKEY", "orderkey"))))); - } - - @Test - public void testTableScan() - { - assertMinimallyOptimizedPlan("SELECT orderkey FROM lineitem", - output(ImmutableList.of("ORDERKEY"), - tableScan("lineitem", ImmutableMap.of("ORDERKEY", "orderkey")))); - } - - @Test - public void testJoinMatcher() - { - assertPlan("SELECT o.orderkey FROM orders o, lineitem l WHERE l.orderkey = o.orderkey", - anyTree( - join(INNER, ImmutableList.of(equiJoinClause("ORDERS_OK", "LINEITEM_OK")), - any( - tableScan("orders").withAlias("ORDERS_OK", columnReference("orders", "orderkey"))), - anyTree( - tableScan("lineitem").withAlias("LINEITEM_OK", columnReference("lineitem", "orderkey")))))); - } - - @Test - public void testSelfJoin() - { - assertPlan("SELECT l.orderkey FROM orders l, orders r WHERE l.orderkey = r.orderkey", - anyTree( - join(INNER, ImmutableList.of(equiJoinClause("L_ORDERS_OK", "R_ORDERS_OK")), - any( - tableScan("orders").withAlias("L_ORDERS_OK", columnReference("orders", "orderkey"))), - anyTree( - tableScan("orders").withAlias("R_ORDERS_OK", columnReference("orders", "orderkey")))))); - } - - @Test - public void testAggregation() - { - assertMinimallyOptimizedPlan("SELECT COUNT(nationkey) FROM nation", - output(ImmutableList.of("COUNT"), - aggregation(ImmutableMap.of("COUNT", functionCall("count", ImmutableList.of("NATIONKEY"))), - tableScan("nation", ImmutableMap.of("NATIONKEY", "nationkey"))))); - } - - @Test - public void testValues() - { - assertMinimallyOptimizedPlan("SELECT * from (VALUES 1, 2)", - output(ImmutableList.of("VALUE"), - values(ImmutableMap.of("VALUE", 0)))); - } - - @Test(expectedExceptions = { IllegalStateException.class }, expectedExceptionsMessageRegExp = ".* doesn't have column .*") - public void testAliasNonexistentColumn() - { - assertMinimallyOptimizedPlan("SELECT orderkey FROM lineitem", - node(OutputNode.class, - node(TableScanNode.class).withAlias("ORDERKEY", columnReference("lineitem", "NXCOLUMN")))); - } - - @Test(expectedExceptions = { IllegalStateException.class }, expectedExceptionsMessageRegExp = "missing expression for alias .*") - public void testReferenceNonexistentAlias() - { - assertMinimallyOptimizedPlan("SELECT orderkey FROM lineitem", - output(ImmutableList.of("NXALIAS"), - tableScan("lineitem", ImmutableMap.of("ORDERKEY", "orderkey")))); - } - - @Test(expectedExceptions = { IllegalStateException.class }, expectedExceptionsMessageRegExp = ".*already bound to expression.*") - public void testDuplicateAliases() - { - assertPlan("SELECT o.orderkey FROM orders o, lineitem l WHERE l.orderkey = o.orderkey", - anyTree( - join(INNER, ImmutableList.of(equiJoinClause("ORDERS_OK", "LINEITEM_OK")), - any( - tableScan("orders").withAlias("ORDERS_OK", columnReference("orders", "orderkey"))), - anyTree( - tableScan("lineitem").withAlias("ORDERS_OK", columnReference("lineitem", "orderkey")))))); - } - - @Test(expectedExceptions = { IllegalStateException.class }, expectedExceptionsMessageRegExp = ".*already bound in.*") - public void testBindMultipleAliasesSameExpression() - { - assertMinimallyOptimizedPlan("SELECT orderkey FROM lineitem", - output(ImmutableList.of("ORDERKEY", "TWO"), - tableScan("lineitem", ImmutableMap.of("FIRST", "orderkey", "SECOND", "orderkey")))); - } - - @Test(expectedExceptions = { IllegalStateException.class }, expectedExceptionsMessageRegExp = "missing expression for alias .*") - public void testProjectLimitsScope() - { - assertMinimallyOptimizedPlan("SELECT 1 + orderkey FROM lineitem", - output(ImmutableList.of("ORDERKEY"), - project(ImmutableMap.of("EXPRESSION", expression("CAST(1 AS bigint) + ORDERKEY")), - tableScan("lineitem", ImmutableMap.of("ORDERKEY", "orderkey"))))); - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestPredicatePushdown.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestPredicatePushdown.java deleted file mode 100644 index 518c425df57d..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestPredicatePushdown.java +++ /dev/null @@ -1,46 +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.sql.planner; - -import com.facebook.presto.sql.planner.assertions.BasePlanTest; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import org.testng.annotations.Test; - -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.any; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.anyTree; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.equiJoinClause; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.filter; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.join; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.tableScan; -import static com.facebook.presto.sql.planner.plan.JoinNode.Type.INNER; - -public class TestPredicatePushdown - extends BasePlanTest -{ - @Test - public void testNonStraddlingJoinExpression() - { - assertPlan("SELECT * FROM orders JOIN lineitem ON orders.orderkey = lineitem.orderkey AND cast(lineitem.linenumber AS varchar) = '2'", - anyTree( - join(INNER, ImmutableList.of(equiJoinClause("ORDERS_OK", "LINEITEM_OK")), - any( - tableScan("orders", ImmutableMap.of("ORDERS_OK", "orderkey"))), - anyTree( - filter("cast(LINEITEM_LINENUMBER as varchar) = cast('2' as varchar)", - tableScan("lineitem", ImmutableMap.of( - "LINEITEM_OK", "orderkey", - "LINEITEM_LINENUMBER", "linenumber"))))))); - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestQuantifiedComparison.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestQuantifiedComparison.java deleted file mode 100644 index a1c687cc4e6f..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestQuantifiedComparison.java +++ /dev/null @@ -1,131 +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.sql.planner; - -import com.facebook.presto.sql.planner.assertions.BasePlanTest; -import com.facebook.presto.sql.planner.plan.AggregationNode; -import com.facebook.presto.sql.planner.plan.JoinNode; -import com.facebook.presto.sql.planner.plan.ValuesNode; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import org.testng.annotations.Test; - -import java.util.Optional; - -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.aggregation; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.anyTree; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.filter; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.functionCall; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.join; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.node; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.project; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.semiJoin; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.tableScan; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.values; -import static com.facebook.presto.sql.planner.plan.JoinNode.Type.INNER; - -public class TestQuantifiedComparison - extends BasePlanTest -{ - @Test - public void testQuantifiedComparisonEqualsAny() - { - String query = "SELECT orderkey, custkey FROM orders WHERE orderkey = ANY (VALUES ROW(CAST(5 as BIGINT)), ROW(CAST(3 as BIGINT)))"; - assertPlan(query, anyTree( - filter("S", - project( - semiJoin("X", "Y", "S", - anyTree(tableScan("orders", ImmutableMap.of("X", "orderkey"))), - anyTree(values(ImmutableMap.of("Y", 0)))))))); - } - - @Test - public void testQuantifiedComparisonNotEqualsAll() - { - String query = "SELECT orderkey, custkey FROM orders WHERE orderkey <> ALL (VALUES ROW(CAST(5 as BIGINT)), ROW(CAST(3 as BIGINT)))"; - assertPlan(query, anyTree( - filter("NOT S", - project( - semiJoin("X", "Y", "S", - anyTree(tableScan("orders", ImmutableMap.of("X", "orderkey"))), - anyTree(values(ImmutableMap.of("Y", 0)))))))); - } - - @Test - public void testQuantifiedComparisonLessAll() - { - assertOrderedQuantifiedComparison("SELECT orderkey, custkey FROM orders WHERE orderkey < ALL (VALUES CAST(5 as BIGINT), CAST(3 as BIGINT))", - "X < MIN", "X", "min", "MIN"); - } - - @Test - public void testQuantifiedComparisonGreaterEqualAll() - { - assertOrderedQuantifiedComparison("SELECT orderkey, custkey FROM orders WHERE orderkey >= ALL (VALUES CAST(5 as BIGINT), CAST(3 as BIGINT))", - "X >= MAX", "X", "max", "MAX"); - } - - @Test - public void testQuantifiedComparisonLessSome() - { - assertOrderedQuantifiedComparison("SELECT orderkey, custkey FROM orders WHERE orderkey < SOME (VALUES CAST(5 as BIGINT), CAST(3 as BIGINT))", - "X < MAX", "X", "max", "MAX"); - } - - @Test - public void testQuantifiedComparisonGreaterEqualAny() - { - assertOrderedQuantifiedComparison("SELECT orderkey, custkey FROM orders WHERE orderkey >= ANY (VALUES CAST(5 as BIGINT), CAST(3 as BIGINT))", - "X >= MIN", "X", "min", "MIN"); - } - - @Test - public void testQuantifiedComparisonEqualAll() - { - String query = "SELECT orderkey, custkey FROM orders WHERE orderkey = ALL (VALUES CAST(5 as BIGINT), CAST(3 as BIGINT))"; - - assertPlan(query, - anyTree( - node(JoinNode.class, - anyTree( - tableScan("orders")), - anyTree( - node(AggregationNode.class, - node(ValuesNode.class)))))); - } - - @Test - public void testQuantifiedComparisonNotEqualAny() - { - String query = "SELECT orderkey, custkey FROM orders WHERE orderkey <> SOME (VALUES CAST(5 as BIGINT), CAST(3 as BIGINT))"; - - assertPlan(query, anyTree( - node(JoinNode.class, - tableScan("orders"), - node(AggregationNode.class, - node(ValuesNode.class))))); - } - - private void assertOrderedQuantifiedComparison(String query, String filter, String columnMapping, String function, String functionAlias) - { - assertPlan(query, anyTree( - project( - join(INNER, ImmutableList.of(), Optional.of(filter), - tableScan("orders", ImmutableMap.of(columnMapping, "orderkey")), - aggregation( - ImmutableMap.of( - functionAlias, functionCall(function, ImmutableList.of("FIELD"))), - values(ImmutableMap.of("FIELD", 0))))))); - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestingTableHandle.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestingTableHandle.java index 1db8042dfa50..aebd02b16fab 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/TestingTableHandle.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/TestingTableHandle.java @@ -14,16 +14,8 @@ package com.facebook.presto.sql.planner; import com.facebook.presto.spi.ConnectorTableHandle; -import com.fasterxml.jackson.annotation.JsonValue; public class TestingTableHandle implements ConnectorTableHandle { - // Jackson refuses to serialize this class otherwise because it's empty. - @JsonValue - @Override - public String toString() - { - return getClass().getSimpleName(); - } } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/AggregationFunctionMatcher.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/AggregationFunctionMatcher.java deleted file mode 100644 index 951f7b13ec18..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/AggregationFunctionMatcher.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.facebook.presto.sql.planner.assertions; - -import com.facebook.presto.Session; -import com.facebook.presto.metadata.Metadata; -import com.facebook.presto.sql.planner.Symbol; -import com.facebook.presto.sql.planner.plan.AggregationNode; -import com.facebook.presto.sql.planner.plan.PlanNode; -import com.facebook.presto.sql.tree.FunctionCall; - -import java.util.Map; -import java.util.Optional; - -import static com.google.common.base.Preconditions.checkState; -import static java.util.Objects.requireNonNull; - -public class AggregationFunctionMatcher - implements RvalueMatcher -{ - private final ExpectedValueProvider callMaker; - - public AggregationFunctionMatcher(ExpectedValueProvider callMaker) - { - this.callMaker = requireNonNull(callMaker, "functionCall is null"); - } - - @Override - public Optional getAssignedSymbol(PlanNode node, Session session, Metadata metadata, SymbolAliases symbolAliases) - { - Optional result = Optional.empty(); - if (!(node instanceof AggregationNode)) { - return result; - } - - AggregationNode aggregationNode = (AggregationNode) node; - - FunctionCall expectedCall = callMaker.getExpectedValue(symbolAliases); - for (Map.Entry assignment : aggregationNode.getAggregations().entrySet()) { - if (expectedCall.equals(assignment.getValue())) { - checkState(!result.isPresent(), "Ambiguous function calls in %s", aggregationNode); - result = Optional.of(assignment.getKey()); - } - } - - return result; - } - - @Override - public String toString() - { - return callMaker.toString(); - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/AggregationMatcher.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/AggregationMatcher.java index 702cb0d30747..5d69e70ca63f 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/AggregationMatcher.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/AggregationMatcher.java @@ -18,6 +18,7 @@ import com.facebook.presto.sql.planner.Symbol; import com.facebook.presto.sql.planner.plan.AggregationNode; import com.facebook.presto.sql.planner.plan.PlanNode; +import com.facebook.presto.sql.tree.FunctionCall; import java.util.Collection; import java.util.List; @@ -25,43 +26,39 @@ import java.util.Optional; import java.util.stream.Collectors; -import static com.facebook.presto.sql.planner.assertions.MatchResult.NO_MATCH; -import static com.facebook.presto.sql.planner.assertions.MatchResult.match; import static com.google.common.base.MoreObjects.toStringHelper; -import static com.google.common.base.Preconditions.checkState; public class AggregationMatcher implements Matcher { + private final List aggregations; private final Map masks; private final List> groupingSets; private final Optional groupId; - public AggregationMatcher(List> groupingSets, Map masks, Optional groupId) + public AggregationMatcher(List> groupingSets, List aggregations, Map masks, Optional groupId) { + this.aggregations = aggregations; this.masks = masks; this.groupingSets = groupingSets; this.groupId = groupId; } @Override - public boolean shapeMatches(PlanNode node) + public boolean matches(PlanNode node, Session session, Metadata metadata, ExpressionAliases expressionAliases) { - return node instanceof AggregationNode; - } + if (!(node instanceof AggregationNode)) { + return false; + } - @Override - public MatchResult detailMatches(PlanNode node, Session session, Metadata metadata, SymbolAliases symbolAliases) - { - checkState(shapeMatches(node), "Plan testing framework error: shapeMatches returned false in detailMatches in %s", this.getClass().getName()); AggregationNode aggregationNode = (AggregationNode) node; if (groupId.isPresent() != aggregationNode.getGroupIdSymbol().isPresent()) { - return NO_MATCH; + return false; } if (groupingSets.size() != aggregationNode.getGroupingSets().size()) { - return NO_MATCH; + return false; } List aggregationsWithMask = aggregationNode.getAggregations() @@ -72,22 +69,26 @@ public MatchResult detailMatches(PlanNode node, Session session, Metadata metada .collect(Collectors.toList()); if (aggregationsWithMask.size() != masks.keySet().size()) { - return NO_MATCH; + return false; } for (Symbol symbol : aggregationsWithMask) { if (!masks.keySet().contains(symbol)) { - return NO_MATCH; + return false; } } for (int i = 0; i < groupingSets.size(); i++) { if (!matches(groupingSets.get(i), aggregationNode.getGroupingSets().get(i))) { - return NO_MATCH; + return false; } } - return match(); + if (!matches(aggregations, aggregationNode.getAggregations().values().stream().collect(Collectors.toList()))) { + return false; + } + + return true; } static boolean matches(Collection expected, Collection actual) @@ -110,6 +111,7 @@ public String toString() { return toStringHelper(this) .add("groupingSets", groupingSets) + .add("aggregations", aggregations) .add("masks", masks) .add("groudId", groupId) .toString(); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/Alias.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/Alias.java deleted file mode 100644 index 81f071ad88ff..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/Alias.java +++ /dev/null @@ -1,68 +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.sql.planner.assertions; - -import com.facebook.presto.Session; -import com.facebook.presto.metadata.Metadata; -import com.facebook.presto.sql.planner.Symbol; -import com.facebook.presto.sql.planner.plan.PlanNode; - -import java.util.Optional; - -import static com.facebook.presto.sql.planner.assertions.MatchResult.match; -import static java.lang.String.format; -import static java.util.Objects.requireNonNull; - -public class Alias - implements Matcher -{ - private final Optional alias; - private final RvalueMatcher matcher; - - Alias(Optional alias, RvalueMatcher matcher) - { - this.alias = requireNonNull(alias, "alias is null"); - this.matcher = requireNonNull(matcher, "matcher is null"); - } - - @Override - public boolean shapeMatches(PlanNode node) - { - return true; - } - - /* - * Aliases must be collected on the way back up the tree for several reasons: - * 1) The rvalue may depend on previously bound aliases (in the case of an - * Expression or FunctionCall) - * 2) Scope: aliases bound in a node are only in scope in nodes higher up - * the tree, just as symbols in a node's output are only in scope in nodes - * higher up the tree. - */ - @Override - public MatchResult detailMatches(PlanNode node, Session session, Metadata metadata, SymbolAliases symbolAliases) - { - Optional symbol = matcher.getAssignedSymbol(node, session, metadata, symbolAliases); - if (symbol.isPresent() && alias.isPresent()) { - return match(alias.get(), symbol.get().toSymbolReference()); - } - return new MatchResult(symbol.isPresent()); - } - - @Override - public String toString() - { - return format("bind %s -> %s", alias, matcher); - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/AnySymbol.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/AnySymbol.java deleted file mode 100644 index 944a171bfecf..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/AnySymbol.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.facebook.presto.sql.planner.assertions; - -import com.facebook.presto.sql.planner.Symbol; -import com.facebook.presto.sql.tree.SymbolReference; - -class AnySymbol - extends Symbol - implements PlanTestSymbol -{ - AnySymbol() - { - super("*"); - } - - @Override - public Symbol toSymbol(SymbolAliases aliases) - { - return this; - } - - @Override - public SymbolReference toSymbolReference() - { - return new AnySymbolReference(); - } - - @Override - public int hashCode() - { - /* - * It is impossible to implement hashCode in a way that is honors the general equals()/hashCode() contract: - * - * 0) If two objects are equal according to the equals(Object) method, then calling the hashCode method on each of the two objects must produce the same integer result. - * 1) AnySymbol.equals() returns true when the other Object passed to it is of type Symbol. - * 2) Assume there is an instance of Symbol s1, such that s1.hashCode() returns h1 - * 3) Assume there is an instance of Symbol s2, such that s2.hashCode() returns h2, where h1 != h2 - * 4) Assume there is an instance of AnySymbol, a1 - * 5) By 1, a1.equals(s1) returns true - * 6) By 1, a1.equals(s1) returns true - * 7) By 0, 2, a1.hashCode() must return h1 - * 8) By 0, 3, a2.hashCode() must return h2 - * 9) We have arrived at a contradiction -> AnySymbol cannot implement hashCode() in a way that honors the general equals/hashCode contract. - * - * In general, overriding hashCode() is unlikely to be useful. Because hashCode() can't be made to work in a way that's consistent with Symbol's hashCode() method, you - * can't put AnySymbol instances in e.g. a HashSet and do a meaningful comparison to a HashSet of Symbol instances if the implementation of HashSet.equals() relies on the - * general equals()/hashCode() contract holding. - * - * If you find a use case for putting AnySymbol in a hash table, feel free to - * implement this. - */ - throw new UnsupportedOperationException("Test object"); - } - - @Override - public boolean equals(Object obj) - { - if (obj == this) { - return true; - } - - if (obj == null) { - return false; - } - - return obj instanceof Symbol; - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/AnySymbolReference.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/AnySymbolReference.java deleted file mode 100644 index 1c35e0a7ad4d..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/AnySymbolReference.java +++ /dev/null @@ -1,50 +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.sql.planner.assertions; - -import com.facebook.presto.sql.tree.SymbolReference; - -class AnySymbolReference - extends SymbolReference -{ - AnySymbolReference() - { - super("*"); - } - - @Override - public int hashCode() - { - /* - * See AnySymbol.hashCode() for an explanation of why this is the way it is. - * If you find a use case for putting AnySymbolReference in a hash table, - * feel free to implement this. - */ - throw new UnsupportedOperationException("Test object"); - } - - @Override - public boolean equals(Object obj) - { - if (obj == this) { - return true; - } - - if (obj == null) { - return false; - } - - return obj instanceof SymbolReference; - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/BasePlanTest.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/BasePlanTest.java deleted file mode 100644 index f38c2c93e112..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/BasePlanTest.java +++ /dev/null @@ -1,114 +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.sql.planner.assertions; - -import com.facebook.presto.sql.analyzer.FeaturesConfig; -import com.facebook.presto.sql.planner.LogicalPlanner; -import com.facebook.presto.sql.planner.Plan; -import com.facebook.presto.sql.planner.optimizations.PlanOptimizer; -import com.facebook.presto.sql.planner.optimizations.PruneIdentityProjections; -import com.facebook.presto.sql.planner.optimizations.PruneUnreferencedOutputs; -import com.facebook.presto.sql.planner.optimizations.UnaliasSymbolReferences; -import com.facebook.presto.testing.LocalQueryRunner; -import com.facebook.presto.tpch.TpchConnectorFactory; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import org.intellij.lang.annotations.Language; - -import java.util.List; - -import static com.facebook.presto.testing.TestingSession.testSessionBuilder; -import static org.testng.Assert.fail; - -public class BasePlanTest -{ - private final LocalQueryRunner queryRunner; - - public BasePlanTest() - { - this.queryRunner = new LocalQueryRunner(testSessionBuilder() - .setCatalog("local") - .setSchema("tiny") - .setSystemProperty("task_concurrency", "1") // these tests don't handle exchanges from local parallel - .build()); - - queryRunner.createCatalog(queryRunner.getDefaultSession().getCatalog().get(), - new TpchConnectorFactory(1), - ImmutableMap.of()); - } - - protected LocalQueryRunner getQueryRunner() - { - return queryRunner; - } - - protected void assertPlan(String sql, PlanMatchPattern pattern) - { - assertPlan(sql, LogicalPlanner.Stage.OPTIMIZED_AND_VALIDATED, pattern); - } - - protected void assertPlan(String sql, LogicalPlanner.Stage stage, PlanMatchPattern pattern) - { - queryRunner.inTransaction(transactionSession -> { - Plan actualPlan = queryRunner.createPlan(transactionSession, sql, stage); - PlanAssert.assertPlan(transactionSession, queryRunner.getMetadata(), actualPlan, pattern); - return null; - }); - } - - protected void assertPlanWithOptimizers(String sql, PlanMatchPattern pattern, List optimizers) - { - queryRunner.inTransaction(transactionSession -> { - FeaturesConfig featuresConfig = new FeaturesConfig() - .setDistributedIndexJoinsEnabled(false) - .setOptimizeHashGeneration(true); - Plan actualPlan = queryRunner.createPlan(transactionSession, sql, optimizers, LogicalPlanner.Stage.OPTIMIZED); - PlanAssert.assertPlan(transactionSession, queryRunner.getMetadata(), actualPlan, pattern); - return null; - }); - } - - protected void assertMinimallyOptimizedPlan(@Language("SQL") String sql, PlanMatchPattern pattern) - { - LocalQueryRunner queryRunner = getQueryRunner(); - FeaturesConfig featuresConfig = new FeaturesConfig() - .setDistributedIndexJoinsEnabled(false) - .setOptimizeHashGeneration(true); - List optimizers = ImmutableList.of( - new UnaliasSymbolReferences(), - new PruneUnreferencedOutputs(), - new PruneIdentityProjections()); - queryRunner.inTransaction(transactionSession -> { - Plan actualPlan = queryRunner.createPlan(transactionSession, sql, optimizers, LogicalPlanner.Stage.OPTIMIZED); - PlanAssert.assertPlan(transactionSession, queryRunner.getMetadata(), actualPlan, pattern); - return null; - }); - } - - protected Plan plan(String sql) - { - return plan(sql, LogicalPlanner.Stage.OPTIMIZED_AND_VALIDATED); - } - - protected Plan plan(String sql, LogicalPlanner.Stage stage) - { - try { - return queryRunner.inTransaction(transactionSession -> queryRunner.createPlan(transactionSession, sql, stage)); - } - catch (RuntimeException ex) { - fail("Invalid SQL: " + sql, ex); - return null; // make compiler happy - } - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ColumnReference.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ColumnReference.java deleted file mode 100644 index afaf6b4b82c9..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ColumnReference.java +++ /dev/null @@ -1,93 +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.sql.planner.assertions; - -import com.facebook.presto.Session; -import com.facebook.presto.metadata.Metadata; -import com.facebook.presto.metadata.TableHandle; -import com.facebook.presto.metadata.TableMetadata; -import com.facebook.presto.spi.ColumnHandle; -import com.facebook.presto.sql.planner.Symbol; -import com.facebook.presto.sql.planner.plan.PlanNode; -import com.facebook.presto.sql.planner.plan.TableScanNode; - -import java.util.Map; -import java.util.Optional; - -import static com.google.common.base.Preconditions.checkState; -import static java.lang.String.format; -import static java.util.Objects.requireNonNull; - -public class ColumnReference - implements RvalueMatcher -{ - private final String tableName; - private final String columnName; - - public ColumnReference(String tableName, String columnName) - { - this.tableName = requireNonNull(tableName, "tableName is null"); - this.columnName = requireNonNull(columnName, "columnName is null"); - } - - @Override - public Optional getAssignedSymbol(PlanNode node, Session session, Metadata metadata, SymbolAliases symbolAliases) - { - if (!(node instanceof TableScanNode)) { - return Optional.empty(); - } - - TableScanNode tableScanNode = (TableScanNode) node; - TableMetadata tableMetadata = metadata.getTableMetadata(session, tableScanNode.getTable()); - String actualTableName = tableMetadata.getTable().getTableName(); - - // Wrong table -> doesn't match. - if (!tableName.equalsIgnoreCase(actualTableName)) { - return Optional.empty(); - } - - Optional columnHandle = getColumnHandle(tableScanNode.getTable(), session, metadata); - - checkState(columnHandle.isPresent(), format("Table %s doesn't have column %s. Typo in test?", tableName, columnName)); - - return getAssignedSymbol(tableScanNode, columnHandle.get()); - } - - private Optional getAssignedSymbol(TableScanNode tableScanNode, ColumnHandle columnHandle) - { - Optional result = Optional.empty(); - for (Map.Entry entry : tableScanNode.getAssignments().entrySet()) { - if (entry.getValue().equals(columnHandle)) { - checkState(!result.isPresent(), "Multiple ColumnHandles found for %s:%s in table scan assignments", tableName, columnName); - result = Optional.of(entry.getKey()); - } - } - return result; - } - - private Optional getColumnHandle(TableHandle tableHandle, Session session, Metadata metadata) - { - return metadata.getColumnHandles(session, tableHandle).entrySet() - .stream() - .filter(entry -> columnName.equals(entry.getKey())) - .map(Map.Entry::getValue) - .findFirst(); - } - - @Override - public String toString() - { - return format("Column %s:%s", tableName, columnName); - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/CorrelationMatcher.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/CorrelationMatcher.java index eb4bf398f8e8..64e3d4d9bb6f 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/CorrelationMatcher.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/CorrelationMatcher.java @@ -21,10 +21,7 @@ import java.util.List; -import static com.facebook.presto.sql.planner.assertions.MatchResult.NO_MATCH; -import static com.facebook.presto.sql.planner.assertions.MatchResult.match; import static com.google.common.base.MoreObjects.toStringHelper; -import static com.google.common.base.Preconditions.checkState; import static java.util.Objects.requireNonNull; public class CorrelationMatcher @@ -32,35 +29,28 @@ public class CorrelationMatcher { private final List correlation; - CorrelationMatcher(List correlation) + public CorrelationMatcher(List correlation) { this.correlation = requireNonNull(correlation, "correlation is null"); } @Override - public boolean shapeMatches(PlanNode node) + public boolean matches(PlanNode node, Session session, Metadata metadata, ExpressionAliases expressionAliases) { - return node instanceof ApplyNode; - } - - @Override - public MatchResult detailMatches(PlanNode node, Session session, Metadata metadata, SymbolAliases symbolAliases) - { - checkState(shapeMatches(node), "Plan testing framework error: shapeMatches returned false in detailMatches in %s", this.getClass().getName()); + if (node instanceof ApplyNode) { + ApplyNode applyNode = (ApplyNode) node; - ApplyNode applyNode = (ApplyNode) node; - - if (correlation.size() != applyNode.getCorrelation().size()) { - return NO_MATCH; - } + if (correlation.size() != applyNode.getCorrelation().size()) { + return false; + } - int i = 0; - for (String alias : correlation) { - if (!symbolAliases.get(alias).equals(applyNode.getCorrelation().get(i++).toSymbolReference())) { - return NO_MATCH; + int i = 0; + for (String alias : correlation) { + expressionAliases.put(alias, applyNode.getCorrelation().get(i++).toSymbolReference()); } + return true; } - return match(); + return false; } @Override diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/EquiJoinClauseProvider.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/EquiJoinClauseProvider.java deleted file mode 100644 index 04a5c350ad27..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/EquiJoinClauseProvider.java +++ /dev/null @@ -1,36 +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.sql.planner.assertions; - -import com.facebook.presto.sql.planner.plan.JoinNode; - -import static java.util.Objects.requireNonNull; - -class EquiJoinClauseProvider - implements ExpectedValueProvider -{ - private final SymbolAlias left; - private final SymbolAlias right; - - EquiJoinClauseProvider(SymbolAlias left, SymbolAlias right) - { - this.left = requireNonNull(left, "left is null"); - this.right = requireNonNull(right, "right is null"); - } - - public JoinNode.EquiJoinClause getExpectedValue(SymbolAliases aliases) - { - return new JoinNode.EquiJoinClause(left.toSymbol(aliases), right.toSymbol(aliases)); - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ExpectedValueProvider.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ExpectedValueProvider.java deleted file mode 100644 index 033754d7705e..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ExpectedValueProvider.java +++ /dev/null @@ -1,33 +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.sql.planner.assertions; - -/** - * An implementation of ExpectedValueProvider should hold the values and - * SymbolAliases needed to call T's constructor, and call - * SymbolAlias.toSymbol() to get actual Symbols to pass to T's constructor. - * Doing this ensures that changes to T's .equals() method that requires a - * change to T's constructor result in a compilation error. - * - * In particular, if adding a new field to T's .equals() method requires - * passing a value for that field to T's constuctor, using an - * ExpectedValueProvider that calls T's constructor will ensure that there is - * a compilation error to be fixed. By contrast, implementing the comparison logic - * in the test code and using SymbolAliases directly will likely cause tests to - * pass erroneously without any notification. - */ -public interface ExpectedValueProvider -{ - T getExpectedValue(SymbolAliases aliases); -} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ExpressionAliases.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ExpressionAliases.java new file mode 100644 index 000000000000..cb6868c344fe --- /dev/null +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ExpressionAliases.java @@ -0,0 +1,71 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.sql.planner.assertions; + +import com.facebook.presto.sql.planner.Symbol; +import com.facebook.presto.sql.tree.Expression; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.ImmutableMultimap; +import com.google.common.collect.Multimap; + +import java.util.Map; + +import static com.google.common.base.Preconditions.checkState; +import static java.util.Objects.requireNonNull; + +public final class ExpressionAliases +{ + private final Multimap map; + + public ExpressionAliases() + { + this.map = ArrayListMultimap.create(); + } + + public ExpressionAliases(ExpressionAliases expressionAliases) + { + requireNonNull(expressionAliases, "symbolAliases are null"); + this.map = ArrayListMultimap.create(expressionAliases.map); + } + + public void put(String alias, Expression expression) + { + alias = alias(alias); + if (map.containsKey(alias)) { + checkState(map.get(alias).contains(expression), "Alias '%s' points to different expression '%s' and '%s'", alias, expression, map.get(alias)); + } + else { + checkState(!map.values().contains(expression), "Expression '%s' is already pointed by different alias than '%s', check mapping for '%s'", expression, alias, map); + map.put(alias, expression); + } + } + + private String alias(String alias) + { + return alias.toLowerCase().replace("(", "").replace(")", "").replace("\"", ""); + } + + public void updateAssignments(Map assignments) + { + ImmutableMultimap.Builder mapUpdate = ImmutableMultimap.builder(); + for (Map.Entry assignment : assignments.entrySet()) { + for (String alias : map.keys()) { + if (map.get(alias).contains(assignment.getKey().toSymbolReference())) { + mapUpdate.put(alias, assignment.getValue()); + } + } + } + map.putAll(mapUpdate.build()); + } +} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ExpressionMatcher.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ExpressionMatcher.java deleted file mode 100644 index 50ac77062e2e..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ExpressionMatcher.java +++ /dev/null @@ -1,99 +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.sql.planner.assertions; - -import com.facebook.presto.Session; -import com.facebook.presto.metadata.Metadata; -import com.facebook.presto.sql.parser.SqlParser; -import com.facebook.presto.sql.planner.Symbol; -import com.facebook.presto.sql.planner.plan.ApplyNode; -import com.facebook.presto.sql.planner.plan.PlanNode; -import com.facebook.presto.sql.planner.plan.ProjectNode; -import com.facebook.presto.sql.tree.Expression; -import com.google.common.collect.ImmutableList; - -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.stream.Collectors; - -import static com.facebook.presto.sql.ExpressionUtils.rewriteQualifiedNamesToSymbolReferences; -import static com.google.common.base.Preconditions.checkState; -import static java.util.Objects.requireNonNull; - -public class ExpressionMatcher - implements RvalueMatcher -{ - private final String sql; - private final Expression expression; - - public ExpressionMatcher(String expression) - { - this.sql = requireNonNull(expression); - this.expression = expression(requireNonNull(expression)); - } - - private Expression expression(String sql) - { - SqlParser parser = new SqlParser(); - return rewriteQualifiedNamesToSymbolReferences(parser.createExpression(sql)); - } - - @Override - public Optional getAssignedSymbol(PlanNode node, Session session, Metadata metadata, SymbolAliases symbolAliases) - { - Optional result = Optional.empty(); - ImmutableList.Builder matchesBuilder = ImmutableList.builder(); - Map assignments = getAssignments(node); - - if (assignments == null) { - return result; - } - - ExpressionVerifier verifier = new ExpressionVerifier(symbolAliases); - - for (Map.Entry assignment : assignments.entrySet()) { - if (verifier.process(assignment.getValue(), expression)) { - result = Optional.of(assignment.getKey()); - matchesBuilder.add(assignment.getValue()); - } - } - - List matches = matchesBuilder.build(); - checkState(matches.size() < 2, "Ambiguous expression %s matches multiple assignments", expression, - (matches.stream().map(Expression::toString).collect(Collectors.joining(", ")))); - return result; - } - - private static Map getAssignments(PlanNode node) - { - if (node instanceof ProjectNode) { - ProjectNode projectNode = (ProjectNode) node; - return projectNode.getAssignments(); - } - else if (node instanceof ApplyNode) { - ApplyNode applyNode = (ApplyNode) node; - return applyNode.getSubqueryAssignments(); - } - else { - return null; - } - } - - @Override - public String toString() - { - return sql; - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ExpressionVerifier.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ExpressionVerifier.java index 1020c38395a9..310d5d7eea56 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ExpressionVerifier.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ExpressionVerifier.java @@ -13,26 +13,19 @@ */ package com.facebook.presto.sql.planner.assertions; -import com.facebook.presto.sql.tree.ArithmeticBinaryExpression; import com.facebook.presto.sql.tree.AstVisitor; -import com.facebook.presto.sql.tree.BooleanLiteral; -import com.facebook.presto.sql.tree.Cast; import com.facebook.presto.sql.tree.ComparisonExpression; -import com.facebook.presto.sql.tree.DoubleLiteral; import com.facebook.presto.sql.tree.Expression; import com.facebook.presto.sql.tree.GenericLiteral; -import com.facebook.presto.sql.tree.InListExpression; import com.facebook.presto.sql.tree.InPredicate; import com.facebook.presto.sql.tree.LogicalBinaryExpression; import com.facebook.presto.sql.tree.LongLiteral; import com.facebook.presto.sql.tree.Node; import com.facebook.presto.sql.tree.NotExpression; +import com.facebook.presto.sql.tree.QualifiedNameReference; import com.facebook.presto.sql.tree.StringLiteral; import com.facebook.presto.sql.tree.SymbolReference; -import java.util.List; - -import static com.google.common.base.Preconditions.checkState; import static java.lang.String.format; import static java.util.Objects.requireNonNull; @@ -63,11 +56,11 @@ final class ExpressionVerifier extends AstVisitor { - private final SymbolAliases symbolAliases; + private final ExpressionAliases expressionAliases; - ExpressionVerifier(SymbolAliases symbolAliases) + ExpressionVerifier(ExpressionAliases expressionAliases) { - this.symbolAliases = requireNonNull(symbolAliases, "symbolAliases is null"); + this.expressionAliases = requireNonNull(expressionAliases, "expressionAliases is null"); } @Override @@ -76,52 +69,12 @@ protected Boolean visitNode(Node node, Expression context) throw new IllegalStateException(format("Node %s is not supported", node)); } - @Override - protected Boolean visitCast(Cast actual, Expression expectedExpression) - { - if (!(expectedExpression instanceof Cast)) { - return false; - } - - Cast expected = (Cast) expectedExpression; - - if (!actual.getType().equals(expected.getType())) { - return false; - } - - return process(actual.getExpression(), expected.getExpression()); - } - @Override protected Boolean visitInPredicate(InPredicate actual, Expression expectedExpression) { if (expectedExpression instanceof InPredicate) { InPredicate expected = (InPredicate) expectedExpression; - - if (actual.getValueList() instanceof InListExpression) { - return process(actual.getValue(), expected.getValue()) && process(actual.getValueList(), expected.getValueList()); - } - else { - checkState( - expected.getValueList() instanceof InListExpression, - "ExpressionVerifier doesn't support unpacked expected values. Feel free to add support if needed"); - /* - * If the expected value is a value list, but the actual is e.g. a SymbolReference, - * we need to unpack the value from the list so that when we hit visitSymbolReference, the - * expected.toString() call returns something that the symbolAliases actually contains. - * For example, InListExpression.toString returns "(onlyitem)" rather than "onlyitem". - * - * This is required because actual passes through the analyzer, planner, and possibly optimizers, - * one of which sometimes takes the liberty of unpacking the InListExpression. - * - * Since the expected value doesn't go through all of that, we have to deal with the case - * of the actual value being unpacked, but the expected value being an InListExpression. - */ - List values = ((InListExpression) expected.getValueList()).getValues(); - checkState(values.size() == 1, "Multiple expressions in expected value list %s, but actual value is not a list", values, actual.getValue()); - Expression onlyExpectedExpression = values.get(0); - return process(actual.getValue(), expected.getValue()) && process(actual.getValueList(), onlyExpectedExpression); - } + return process(actual.getValue(), expected.getValue()) && process(actual.getValueList(), expected.getValueList()); } return false; } @@ -138,18 +91,6 @@ protected Boolean visitComparisonExpression(ComparisonExpression actual, Express return false; } - @Override - protected Boolean visitArithmeticBinary(ArithmeticBinaryExpression actual, Expression expectedExpression) - { - if (expectedExpression instanceof ArithmeticBinaryExpression) { - ArithmeticBinaryExpression expected = (ArithmeticBinaryExpression) expectedExpression; - if (actual.getType() == expected.getType()) { - return process(actual.getLeft(), expected.getLeft()) && process(actual.getRight(), expected.getRight()); - } - } - return false; - } - protected Boolean visitGenericLiteral(GenericLiteral actual, Expression expected) { return getValueFromLiteral(actual).equals(getValueFromLiteral(expected)); @@ -161,33 +102,11 @@ protected Boolean visitLongLiteral(LongLiteral actual, Expression expected) return getValueFromLiteral(actual).equals(getValueFromLiteral(expected)); } - @Override - protected Boolean visitDoubleLiteral(DoubleLiteral actual, Expression expected) - { - if (expected instanceof DoubleLiteral) { - return getValueFromLiteral(actual).equals(getValueFromLiteral(expected)); - } - - return false; - } - - @Override - protected Boolean visitBooleanLiteral(BooleanLiteral actual, Expression expected) - { - return getValueFromLiteral(actual).equals(getValueFromLiteral(expected)); - } - - private static String getValueFromLiteral(Expression expression) + private String getValueFromLiteral(Expression expression) { if (expression instanceof LongLiteral) { return String.valueOf(((LongLiteral) expression).getValue()); } - else if (expression instanceof BooleanLiteral) { - return String.valueOf(((BooleanLiteral) expression).getValue()); - } - else if (expression instanceof DoubleLiteral) { - return String.valueOf(((DoubleLiteral) expression).getValue()); - } else if (expression instanceof GenericLiteral) { return ((GenericLiteral) expression).getValue(); } @@ -227,12 +146,17 @@ protected Boolean visitNotExpression(NotExpression actual, Expression expected) return false; } + @Override + protected Boolean visitQualifiedNameReference(QualifiedNameReference actual, Expression expected) + { + expressionAliases.put(expected.toString(), actual); + return true; + } + @Override protected Boolean visitSymbolReference(SymbolReference actual, Expression expected) { - if (!(expected instanceof SymbolReference)) { - return false; - } - return symbolAliases.get(((SymbolReference) expected).getName()).equals(actual); + expressionAliases.put(expected.toString(), actual); + return true; } } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/FilterMatcher.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/FilterMatcher.java index 8041f618e16b..264396f8f4de 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/FilterMatcher.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/FilterMatcher.java @@ -21,7 +21,6 @@ import com.facebook.presto.sql.tree.Expression; import static com.google.common.base.MoreObjects.toStringHelper; -import static com.google.common.base.Preconditions.checkState; import static java.util.Objects.requireNonNull; final class FilterMatcher @@ -35,19 +34,15 @@ final class FilterMatcher } @Override - public boolean shapeMatches(PlanNode node) + public boolean matches(PlanNode node, Session session, Metadata metadata, ExpressionAliases expressionAliases) { - return node instanceof FilterNode; - } - - @Override - public MatchResult detailMatches(PlanNode node, Session session, Metadata metadata, SymbolAliases symbolAliases) - { - checkState(shapeMatches(node), "Plan testing framework error: shapeMatches returned false in detailMatches in %s", this.getClass().getName()); - - FilterNode filterNode = (FilterNode) node; - ExpressionVerifier verifier = new ExpressionVerifier(symbolAliases); - return new MatchResult(verifier.process(filterNode.getPredicate(), predicate)); + if (node instanceof FilterNode) { + FilterNode filterNode = (FilterNode) node; + if (new ExpressionVerifier(expressionAliases).process(filterNode.getPredicate(), predicate)) { + return true; + } + } + return false; } @Override diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/FunctionCallProvider.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/FunctionCallProvider.java deleted file mode 100644 index ec980b4ca92e..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/FunctionCallProvider.java +++ /dev/null @@ -1,122 +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.sql.planner.assertions; - -import com.facebook.presto.sql.tree.Expression; -import com.facebook.presto.sql.tree.FunctionCall; -import com.facebook.presto.sql.tree.QualifiedName; -import com.facebook.presto.sql.tree.WindowFrame; -import com.google.common.base.Joiner; - -import java.util.List; -import java.util.Objects; -import java.util.Optional; - -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.toSymbolReferences; -import static java.lang.String.format; -import static java.util.Objects.requireNonNull; - -class FunctionCallProvider - implements ExpectedValueProvider -{ - private boolean isWindowFunction; - private final QualifiedName name; - private final Optional frame; - private final boolean distinct; - private final List args; - - private FunctionCallProvider(boolean isWindowFunction, QualifiedName name, Optional frame, boolean distinct, List args) - { - this.isWindowFunction = isWindowFunction; - this.name = requireNonNull(name, "name is null"); - this.frame = requireNonNull(frame, "frame is null"); - this.distinct = distinct; - this.args = requireNonNull(args, "args is null"); - } - - FunctionCallProvider(QualifiedName name, Optional frame, boolean distinct, List args) - { - this(true, name, frame, distinct, args); - } - - FunctionCallProvider(QualifiedName name, boolean distinct, List args) - { - this(false, name, Optional.empty(), distinct, args); - } - - FunctionCallProvider(QualifiedName name, List args) - { - this(false, name, Optional.empty(), false, args); - } - - @Override - public String toString() - { - return format("%s%s (%s) %s", distinct ? "DISTINCT" : "", name, Joiner.on(", ").join(args), frame.isPresent() ? frame.get().toString() : ""); - } - - public FunctionCall getExpectedValue(SymbolAliases aliases) - { - List symbolReferences = toSymbolReferences(args, aliases); - if (isWindowFunction) { - return new ExpectedWindowFunctionCall(symbolReferences); - } - - return new FunctionCall(name, symbolReferences); - } - - private class ExpectedWindowFunctionCall - extends FunctionCall - { - private ExpectedWindowFunctionCall(List args) - { - super(name, distinct, args); - } - - @Override - public boolean equals(Object object) - { - if (this == object) { - return true; - } - - if (object == null || !(object instanceof FunctionCall)) { - return false; - } - - FunctionCall other = (FunctionCall) object; - - return Objects.equals(name, other.getName()) && - other.getWindow().isPresent() && - Objects.equals(frame, other.getWindow().get().getFrame()) && - Objects.equals(distinct, other.isDistinct()) && - Objects.equals(getArguments(), other.getArguments()); - } - - @Override - public int hashCode() - { - /* - * Putting this in a hash table is probably not a useful thing to do, - * especially not if you want to compare this with an actual WindowFunction. - * This is because (by necessity) ExpectedWindowFunctionCalls don't have the - * same fields as FunctionCalls, and can't hash the same as a result. - * - * If you find a useful case for putting this in a hash table, feel free to - * add an implementation. Until then, it would just be dead and untested code. - */ - throw new UnsupportedOperationException("Test object"); - } - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/GroupIdMatcher.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/GroupIdMatcher.java index 7f750d6c0efe..907a3c132dda 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/GroupIdMatcher.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/GroupIdMatcher.java @@ -22,10 +22,7 @@ import java.util.List; import java.util.Map; -import static com.facebook.presto.sql.planner.assertions.MatchResult.NO_MATCH; -import static com.facebook.presto.sql.planner.assertions.MatchResult.match; import static com.google.common.base.MoreObjects.toStringHelper; -import static com.google.common.base.Preconditions.checkState; public class GroupIdMatcher implements Matcher @@ -40,35 +37,31 @@ public GroupIdMatcher(List> groups, Map identityMap } @Override - public boolean shapeMatches(PlanNode node) + public boolean matches(PlanNode node, Session session, Metadata metadata, ExpressionAliases expressionAliases) { - return node instanceof GroupIdNode; - } - - @Override - public MatchResult detailMatches(PlanNode node, Session session, Metadata metadata, SymbolAliases symbolAliases) - { - checkState(shapeMatches(node), "Plan testing framework error: shapeMatches returned false in detailMatches in %s", this.getClass().getName()); + if (!(node instanceof GroupIdNode)) { + return false; + } GroupIdNode groudIdNode = (GroupIdNode) node; List> actualGroups = groudIdNode.getGroupingSets(); Map actualArgumentMappings = groudIdNode.getArgumentMappings(); if (actualGroups.size() != groups.size()) { - return NO_MATCH; + return false; } for (int i = 0; i < actualGroups.size(); i++) { if (!AggregationMatcher.matches(actualGroups.get(i), groups.get(i))) { - return NO_MATCH; + return false; } } if (!AggregationMatcher.matches(identityMappings.keySet(), actualArgumentMappings.keySet())) { - return NO_MATCH; + return false; } - return match(); + return true; } @Override diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/JoinMatcher.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/JoinMatcher.java index f08a257d2d35..08745f338109 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/JoinMatcher.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/JoinMatcher.java @@ -18,80 +18,43 @@ import com.facebook.presto.metadata.Metadata; import com.facebook.presto.sql.planner.plan.JoinNode; import com.facebook.presto.sql.planner.plan.PlanNode; -import com.facebook.presto.sql.tree.Expression; -import com.google.common.collect.ImmutableSet; import java.util.List; -import java.util.Optional; -import java.util.Set; -import static com.facebook.presto.sql.planner.assertions.MatchResult.NO_MATCH; -import static com.facebook.presto.util.ImmutableCollectors.toImmutableSet; import static com.google.common.base.MoreObjects.toStringHelper; -import static com.google.common.base.Preconditions.checkState; import static java.util.Objects.requireNonNull; final class JoinMatcher implements Matcher { private final JoinNode.Type joinType; - private final List> equiCriteria; - private final Optional filter; + private final List equiCriteria; - JoinMatcher(JoinNode.Type joinType, List> equiCriteria, Optional filter) + JoinMatcher(JoinNode.Type joinType, List equiCriteria) { this.joinType = requireNonNull(joinType, "joinType is null"); this.equiCriteria = requireNonNull(equiCriteria, "equiCriteria is null"); - this.filter = requireNonNull(filter, "filter can not be null"); } @Override - public boolean shapeMatches(PlanNode node) + public boolean matches(PlanNode node, Session session, Metadata metadata, ExpressionAliases expressionAliases) { - if (!(node instanceof JoinNode)) { - return false; - } - - JoinNode joinNode = (JoinNode) node; - return joinNode.getType() == joinType; - } - - @Override - public MatchResult detailMatches(PlanNode node, Session session, Metadata metadata, SymbolAliases symbolAliases) - { - checkState(shapeMatches(node), "Plan testing framework error: shapeMatches returned false in detailMatches in %s", this.getClass().getName()); - - JoinNode joinNode = (JoinNode) node; - - if (joinNode.getCriteria().size() != equiCriteria.size()) { - return NO_MATCH; - } - - if (filter.isPresent()) { - if (!joinNode.getFilter().isPresent()) { - return NO_MATCH; + if (node instanceof JoinNode) { + JoinNode joinNode = (JoinNode) node; + if (joinNode.getType() != joinType) { + return false; } - if (!new ExpressionVerifier(symbolAliases).process(joinNode.getFilter().get(), filter.get())) { - return NO_MATCH; + if (joinNode.getCriteria().size() == equiCriteria.size()) { + int i = 0; + for (JoinNode.EquiJoinClause equiJoinClause : joinNode.getCriteria()) { + AliasPair expectedEquiClause = equiCriteria.get(i++); + expressionAliases.put(expectedEquiClause.left, equiJoinClause.getLeft().toSymbolReference()); + expressionAliases.put(expectedEquiClause.right, equiJoinClause.getRight().toSymbolReference()); + } + return true; } } - else { - if (joinNode.getFilter().isPresent()) { - return NO_MATCH; - } - } - - /* - * Have to use order-independent comparison; there are no guarantees what order - * the equi criteria will have after planning and optimizing. - */ - Set actual = ImmutableSet.copyOf(joinNode.getCriteria()); - Set expected = - equiCriteria.stream() - .map(maker -> maker.getExpectedValue(symbolAliases)) - .collect(toImmutableSet()); - - return new MatchResult(expected.equals(actual)); + return false; } @Override @@ -99,7 +62,6 @@ public String toString() { return toStringHelper(this) .add("equiCriteria", equiCriteria) - .add("filter", filter) .toString(); } } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/MatchResult.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/MatchResult.java deleted file mode 100644 index 8556314eb233..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/MatchResult.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.facebook.presto.sql.planner.assertions; - -import com.facebook.presto.sql.tree.SymbolReference; - -import static java.util.Objects.requireNonNull; - -public class MatchResult -{ - public static final MatchResult NO_MATCH = new MatchResult(false, new SymbolAliases()); - - private final boolean matches; - private final SymbolAliases newAliases; - - public static MatchResult match() - { - return new MatchResult(true, new SymbolAliases()); - } - - public static MatchResult match(String alias, SymbolReference symbolReference) - { - SymbolAliases newAliases = SymbolAliases.builder() - .put(alias, symbolReference) - .build(); - return new MatchResult(true, newAliases); - } - - public static MatchResult match(SymbolAliases newAliases) - { - return new MatchResult(true, newAliases); - } - - public MatchResult(boolean matches) - { - this(matches, new SymbolAliases()); - } - - private MatchResult(boolean matches, SymbolAliases newAliases) - { - this.matches = matches; - this.newAliases = requireNonNull(newAliases, "newAliases is null"); - } - - public boolean isMatch() - { - return matches; - } - - public SymbolAliases getAliases() - { - return newAliases; - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/Matcher.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/Matcher.java index ff4b6deefaa5..30a5ebed2575 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/Matcher.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/Matcher.java @@ -19,53 +19,5 @@ public interface Matcher { - /** - * Verifies that the PlanNode passes basic matching tests that can done - * with only the information contained in the node itself. Typically, these - * should be limited to tests that validate the type of the node or - * attributes of that type. - *

- * Matchers that can be applied to nodes of any typeshould return true from - * shapeMatches and do the rest of their work in detailMatches. - * - * @param node The node to apply the matching tests to - * @return true if all matching tests pass, false otherwise - */ - boolean shapeMatches(PlanNode node); - - /** - * Verifies that the Plan node passes in-depth matching tests. Matching - * tests that check detailed information in a node's internals should be in - * detailMatches. In particular, matching tests that need to reference symbol - * aliases from source nodes must be in a Matcher's detailMatches method. - *

- * The detailMatches method may collect Symbol aliases from the node that it is - * being applied to, and return them in the MatchResult it returns. - * detailMatches must ONLY collect SymbolAliases that are new to the node it is - * being applied to. Specifically, the MatchResult returned by - * detailMatches MUST NOT contain any of the aliases contained in the - * SymbolAliases that was passed in to detailMatches(). - *

- * This is because the caller of detailMatches is responsible for calling - * detailMatches for all of the source nodes/patterns, and then returning the - * union of all of they symbols they returned to be used when applying the - * parent nodes Matchers. If two Matchers each added their source aliases - * to their results, the caller would see duplicate aliases while computing - * the union of the returned aliases. - *

- * Matchers that don't need to validate anything about the internals of a - * node should return a MatchResult with true and an empty - * SymbolAliases object from detailMatches and do all of their work in - * shapeMatches. - *

- * The plan testing framework should not call a Matcher's detailMatches on a - * node if shapeMatches didn't return true for the same node. - * - * @param node The node to apply the matching tests to - * @param session The session information for the query - * @param metadata The metadata for the query - * @param symbolAliases The SymbolAliases containing aliases from the nodes sources - * @return a MatchResult with information about the success of the match - */ - MatchResult detailMatches(PlanNode node, Session session, Metadata metadata, SymbolAliases symbolAliases); + boolean matches(PlanNode node, Session session, Metadata metadata, ExpressionAliases expressionAliases); } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/NotPlanNodeMatcher.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/NotPlanNodeMatcher.java index 268327dc9f3a..b1c1733eb374 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/NotPlanNodeMatcher.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/NotPlanNodeMatcher.java @@ -17,9 +17,7 @@ import com.facebook.presto.metadata.Metadata; import com.facebook.presto.sql.planner.plan.PlanNode; -import static com.facebook.presto.sql.planner.assertions.MatchResult.match; import static com.google.common.base.MoreObjects.toStringHelper; -import static com.google.common.base.Preconditions.checkState; import static java.util.Objects.requireNonNull; final class NotPlanNodeMatcher implements Matcher @@ -32,18 +30,11 @@ final class NotPlanNodeMatcher implements Matcher } @Override - public boolean shapeMatches(PlanNode node) + public boolean matches(PlanNode node, Session session, Metadata metadata, ExpressionAliases expressionAliases) { return (!node.getClass().equals(excludedNodeClass)); } - @Override - public MatchResult detailMatches(PlanNode node, Session session, Metadata metadata, SymbolAliases symbolAliases) - { - checkState(shapeMatches(node), "Plan testing framework error: shapeMatches returned false in detailMatches in %s", this.getClass().getName()); - return match(); - } - @Override public String toString() { diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/OutputMatcher.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/OutputMatcher.java deleted file mode 100644 index 0f00124bf72f..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/OutputMatcher.java +++ /dev/null @@ -1,74 +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.sql.planner.assertions; - -import com.facebook.presto.Session; -import com.facebook.presto.metadata.Metadata; -import com.facebook.presto.sql.planner.Symbol; -import com.facebook.presto.sql.planner.plan.PlanNode; -import com.facebook.presto.sql.tree.Expression; -import com.google.common.collect.ImmutableList; - -import java.util.List; - -import static com.facebook.presto.sql.planner.assertions.MatchResult.NO_MATCH; -import static com.facebook.presto.sql.planner.assertions.MatchResult.match; -import static com.google.common.base.MoreObjects.toStringHelper; -import static java.util.Objects.requireNonNull; - -public class OutputMatcher - implements Matcher -{ - private final List aliases; - - OutputMatcher(List aliases) - { - this.aliases = ImmutableList.copyOf(requireNonNull(aliases, "aliases is null")); - } - - @Override - public boolean shapeMatches(PlanNode node) - { - return true; - } - - @Override - public MatchResult detailMatches(PlanNode node, Session session, Metadata metadata, SymbolAliases symbolAliases) - { - int i = 0; - for (String alias : aliases) { - Expression expression = symbolAliases.get(alias); - boolean found = false; - while (i < node.getOutputSymbols().size()) { - Symbol outputSymbol = node.getOutputSymbols().get(i++); - if (expression.equals(outputSymbol.toSymbolReference())) { - found = true; - break; - } - } - if (!found) { - return NO_MATCH; - } - } - return match(); - } - - @Override - public String toString() - { - return toStringHelper(this) - .add("outputs", aliases) - .toString(); - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanAssert.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanAssert.java index 8d354f0d9443..5708beb40812 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanAssert.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanAssert.java @@ -19,6 +19,7 @@ import static com.facebook.presto.sql.planner.PlanPrinter.textLogicalPlan; import static java.lang.String.format; +import static java.util.Objects.requireNonNull; import static org.testng.Assert.assertTrue; public final class PlanAssert @@ -27,10 +28,12 @@ private PlanAssert() {} public static void assertPlan(Session session, Metadata metadata, Plan actual, PlanMatchPattern pattern) { - MatchResult matches = actual.getRoot().accept(new PlanMatchingVisitor(session, metadata), pattern); - if (!matches.isMatch()) { + requireNonNull(actual, "root is null"); + + boolean matches = actual.getRoot().accept(new PlanMatchingVisitor(session, metadata), new PlanMatchingContext(pattern)); + if (!matches) { String logicalPlan = textLogicalPlan(actual.getRoot(), actual.getTypes(), metadata, session); - assertTrue(matches.isMatch(), format("Plan does not match:\n %s\n, to pattern:\n%s", logicalPlan, pattern)); + assertTrue(matches, format("Plan does not match:\n %s\n, to pattern:\n%s", logicalPlan, pattern)); } } } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanMatchPattern.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanMatchPattern.java index 41de270b0d36..5583e7a15240 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanMatchPattern.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanMatchPattern.java @@ -15,29 +15,24 @@ import com.facebook.presto.Session; import com.facebook.presto.metadata.Metadata; -import com.facebook.presto.spi.block.SortOrder; import com.facebook.presto.spi.predicate.Domain; import com.facebook.presto.sql.parser.SqlParser; import com.facebook.presto.sql.planner.Symbol; import com.facebook.presto.sql.planner.plan.AggregationNode; import com.facebook.presto.sql.planner.plan.ApplyNode; -import com.facebook.presto.sql.planner.plan.ExceptNode; import com.facebook.presto.sql.planner.plan.FilterNode; import com.facebook.presto.sql.planner.plan.GroupIdNode; -import com.facebook.presto.sql.planner.plan.IntersectNode; import com.facebook.presto.sql.planner.plan.JoinNode; -import com.facebook.presto.sql.planner.plan.OutputNode; import com.facebook.presto.sql.planner.plan.PlanNode; import com.facebook.presto.sql.planner.plan.ProjectNode; import com.facebook.presto.sql.planner.plan.SemiJoinNode; import com.facebook.presto.sql.planner.plan.TableScanNode; -import com.facebook.presto.sql.planner.plan.UnionNode; -import com.facebook.presto.sql.planner.plan.ValuesNode; import com.facebook.presto.sql.planner.plan.WindowNode; import com.facebook.presto.sql.tree.Expression; import com.facebook.presto.sql.tree.FunctionCall; import com.facebook.presto.sql.tree.QualifiedName; -import com.facebook.presto.sql.tree.WindowFrame; +import com.facebook.presto.sql.tree.SymbolReference; +import com.facebook.presto.sql.tree.Window; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -45,13 +40,10 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; -import static com.facebook.presto.sql.ExpressionUtils.rewriteQualifiedNamesToSymbolReferences; -import static com.facebook.presto.sql.planner.assertions.MatchResult.NO_MATCH; -import static com.facebook.presto.sql.planner.assertions.MatchResult.match; import static com.facebook.presto.util.ImmutableCollectors.toImmutableList; -import static com.facebook.presto.util.ImmutableCollectors.toImmutableMap; import static com.google.common.base.Preconditions.checkState; import static java.util.Collections.nCopies; import static java.util.Objects.requireNonNull; @@ -93,85 +85,14 @@ public static PlanMatchPattern tableScan(String expectedTableName) return node(TableScanNode.class).with(new TableScanMatcher(expectedTableName)); } - public static PlanMatchPattern tableScan(String expectedTableName, Map columnReferences) - { - PlanMatchPattern result = tableScan(expectedTableName); - return result.addColumnReferences(expectedTableName, columnReferences); - } - - public static PlanMatchPattern constrainedTableScan(String expectedTableName, Map constraint) + public static PlanMatchPattern tableScan(String expectedTableName, Map constraint) { return node(TableScanNode.class).with(new TableScanMatcher(expectedTableName, constraint)); } - public static PlanMatchPattern constrainedTableScan(String expectedTableName, Map constraint, Map columnReferences) - { - PlanMatchPattern result = constrainedTableScan(expectedTableName, constraint); - return result.addColumnReferences(expectedTableName, columnReferences); - } - - private PlanMatchPattern addColumnReferences(String expectedTableName, Map columnReferences) - { - columnReferences.entrySet().forEach( - reference -> withAlias(reference.getKey(), columnReference(expectedTableName, reference.getValue()))); - return this; - } - - public static PlanMatchPattern aggregation( - Map> aggregations, - PlanMatchPattern source) - { - PlanMatchPattern result = node(AggregationNode.class, source); - aggregations.entrySet().forEach( - aggregation -> result.withAlias(aggregation.getKey(), new AggregationFunctionMatcher(aggregation.getValue()))); - return result; - } - - public static PlanMatchPattern aggregation( - List> groupingSets, - Map, ExpectedValueProvider> aggregations, - Map masks, - Optional groupId, - PlanMatchPattern source) + public static PlanMatchPattern window(List functionCalls, PlanMatchPattern source) { - PlanMatchPattern result = node(AggregationNode.class, source).with(new AggregationMatcher(groupingSets, masks, groupId)); - aggregations.entrySet().forEach( - aggregation -> result.withAlias(aggregation.getKey(), new AggregationFunctionMatcher(aggregation.getValue()))); - return result; - } - - public static PlanMatchPattern window( - ExpectedValueProvider specification, - List> windowFunctions, - PlanMatchPattern source) - { - PlanMatchPattern result = node(WindowNode.class, source).with(new WindowMatcher(specification)); - windowFunctions.forEach( - function -> result.withAlias(Optional.empty(), new WindowFunctionMatcher(function))); - return result; - } - - public static PlanMatchPattern window( - ExpectedValueProvider specification, - Map> assignments, - PlanMatchPattern source) - { - PlanMatchPattern result = node(WindowNode.class, source).with(new WindowMatcher(specification)); - assignments.entrySet().forEach( - assignment -> result.withAlias(assignment.getKey(), new WindowFunctionMatcher(assignment.getValue()))); - return result; - } - - public static PlanMatchPattern output(PlanMatchPattern source) - { - return node(OutputNode.class, source); - } - - public static PlanMatchPattern output(List outputs, PlanMatchPattern source) - { - PlanMatchPattern result = output(source); - result.withOutputs(outputs); - return result; + return node(WindowNode.class, source).with(new WindowMatcher(functionCalls)); } public static PlanMatchPattern project(PlanMatchPattern source) @@ -179,71 +100,30 @@ public static PlanMatchPattern project(PlanMatchPattern source) return node(ProjectNode.class, source); } - public static PlanMatchPattern project(Map assignments, PlanMatchPattern source) - { - PlanMatchPattern result = project(source); - assignments.entrySet().forEach( - assignment -> result.withAlias(assignment.getKey(), assignment.getValue())); - return result; - } - public static PlanMatchPattern semiJoin(String sourceSymbolAlias, String filteringSymbolAlias, String outputAlias, PlanMatchPattern source, PlanMatchPattern filtering) { return node(SemiJoinNode.class, source, filtering).with(new SemiJoinMatcher(sourceSymbolAlias, filteringSymbolAlias, outputAlias)); } - public static PlanMatchPattern join(JoinNode.Type joinType, List> expectedEquiCriteria, PlanMatchPattern left, PlanMatchPattern right) - { - return join(joinType, expectedEquiCriteria, Optional.empty(), left, right); - } - - public static PlanMatchPattern join(JoinNode.Type joinType, List> expectedEquiCriteria, Optional expectedFilter, PlanMatchPattern left, PlanMatchPattern right) - { - return node(JoinNode.class, left, right).with( - new JoinMatcher( - joinType, - expectedEquiCriteria, - expectedFilter.map(predicate -> rewriteQualifiedNamesToSymbolReferences(new SqlParser().createExpression(predicate))))); - } - - public static PlanMatchPattern union(PlanMatchPattern... sources) - { - return node(UnionNode.class, sources); - } - - public static PlanMatchPattern intersect(PlanMatchPattern... sources) - { - return node(IntersectNode.class, sources); - } - - public static PlanMatchPattern except(PlanMatchPattern... sources) - { - return node(ExceptNode.class, sources); - } - - public static ExpectedValueProvider equiJoinClause(String left, String right) + public static PlanMatchPattern join(JoinNode.Type joinType, List expectedEquiCriteria, PlanMatchPattern left, PlanMatchPattern right) { - return new EquiJoinClauseProvider(new SymbolAlias(left), new SymbolAlias(right)); + return node(JoinNode.class, left, right).with(new JoinMatcher(joinType, expectedEquiCriteria)); } - public static SymbolAlias symbol(String alias) + public static AliasPair aliasPair(String left, String right) { - return new SymbolAlias(alias); + return new AliasPair(left, right); } public static PlanMatchPattern filter(String predicate, PlanMatchPattern source) { - Expression expectedPredicate = rewriteQualifiedNamesToSymbolReferences(new SqlParser().createExpression(predicate)); + Expression expectedPredicate = new SqlParser().createExpression(predicate); return node(FilterNode.class, source).with(new FilterMatcher(expectedPredicate)); } - public static PlanMatchPattern apply(List correlationSymbolAliases, Map subqueryAssignments, PlanMatchPattern inputPattern, PlanMatchPattern subqueryPattern) + public static PlanMatchPattern apply(List correlationSymbolAliases, PlanMatchPattern inputPattern, PlanMatchPattern subqueryPattern) { - PlanMatchPattern result = node(ApplyNode.class, inputPattern, subqueryPattern) - .with(new CorrelationMatcher(correlationSymbolAliases)); - subqueryAssignments.entrySet().forEach( - assignment -> result.withAlias(assignment.getKey(), assignment.getValue())); - return result; + return node(ApplyNode.class, inputPattern, subqueryPattern).with(new CorrelationMatcher(correlationSymbolAliases)); } public static PlanMatchPattern groupingSet(List> groups, PlanMatchPattern source) @@ -251,12 +131,9 @@ public static PlanMatchPattern groupingSet(List> groups, PlanMatchP return node(GroupIdNode.class, source).with(new GroupIdMatcher(groups, ImmutableMap.of())); } - public static PlanMatchPattern values(Map values) + public static PlanMatchPattern aggregation(List> groupingSets, List aggregations, Map masks, Optional groupId, PlanMatchPattern source) { - PlanMatchPattern result = node(ValuesNode.class); - values.entrySet().forEach( - alias -> result.withAlias(alias.getKey(), new ValuesMatcher(alias.getValue()))); - return result; + return node(AggregationNode.class, source).with(new AggregationMatcher(groupingSets, aggregations, masks, groupId)); } public PlanMatchPattern(List sourcePatterns) @@ -266,37 +143,27 @@ public PlanMatchPattern(List sourcePatterns) this.sourcePatterns = ImmutableList.copyOf(sourcePatterns); } - List shapeMatches(PlanNode node) + List matches(PlanNode node, Session session, Metadata metadata, ExpressionAliases expressionAliases) { ImmutableList.Builder states = ImmutableList.builder(); if (anyTree) { int sourcesCount = node.getSources().size(); if (sourcesCount > 1) { - states.add(new PlanMatchingState(nCopies(sourcesCount, this))); + states.add(new PlanMatchingState(nCopies(sourcesCount, this), expressionAliases)); } else { - states.add(new PlanMatchingState(ImmutableList.of(this))); + states.add(new PlanMatchingState(ImmutableList.of(this), expressionAliases)); } } - if (node.getSources().size() == sourcePatterns.size() && matchers.stream().allMatch(it -> it.shapeMatches(node))) { - states.add(new PlanMatchingState(sourcePatterns)); + if (node.getSources().size() == sourcePatterns.size() && matchers.stream().allMatch(it -> it.matches(node, session, metadata, expressionAliases))) { + states.add(new PlanMatchingState(sourcePatterns, expressionAliases)); } return states.build(); } - MatchResult detailMatches(PlanNode node, Session session, Metadata metadata, SymbolAliases symbolAliases) + public PlanMatchPattern withSymbol(String pattern, String alias) { - SymbolAliases.Builder newAliases = SymbolAliases.builder(); - - for (Matcher matcher : matchers) { - MatchResult matchResult = matcher.detailMatches(node, session, metadata, symbolAliases); - if (!matchResult.isMatch()) { - return NO_MATCH; - } - newAliases.putAll(matchResult.getAliases()); - } - - return match(newAliases.build()); + return with(new SymbolMatcher(pattern, alias)); } public PlanMatchPattern with(Matcher matcher) @@ -305,33 +172,6 @@ public PlanMatchPattern with(Matcher matcher) return this; } - public PlanMatchPattern withAlias(String alias, RvalueMatcher matcher) - { - return withAlias(Optional.of(alias), matcher); - } - - public PlanMatchPattern withAlias(Optional alias, RvalueMatcher matcher) - { - matchers.add(new Alias(alias, matcher)); - return this; - } - - public static RvalueMatcher columnReference(String tableName, String columnName) - { - return new ColumnReference(tableName, columnName); - } - - public static ExpressionMatcher expression(String expression) - { - return new ExpressionMatcher(expression); - } - - public PlanMatchPattern withOutputs(List aliases) - { - matchers.add(new OutputMatcher(aliases)); - return this; - } - public PlanMatchPattern matchToAnyNodeTree() { anyTree = true; @@ -343,66 +183,29 @@ public boolean isTerminated() return sourcePatterns.isEmpty(); } - public static PlanTestSymbol anySymbol() - { - return new AnySymbol(); - } - - public static ExpectedValueProvider functionCall(String name, List args) + private static List toExpressionList(String... args) { - return new FunctionCallProvider(QualifiedName.of(name), toSymbolAliases(args)); - } - - public static ExpectedValueProvider functionCall( - String name, - Optional frame, - List args) - { - return new FunctionCallProvider(QualifiedName.of(name), frame, false, toSymbolAliases(args)); - } + ImmutableList.Builder builder = ImmutableList.builder(); + for (String arg : args) { + if (arg.equals("*")) { + builder.add(new PlanMatchPattern.AnySymbolReference()); + } + else { + builder.add(new SymbolReference(arg)); + } + } - public static ExpectedValueProvider functionCall( - String name, - boolean distinct, - List args) - { - return new FunctionCallProvider(QualifiedName.of(name), distinct, args); + return builder.build(); } - public static List toSymbolReferences(List aliases, SymbolAliases symbolAliases) + public static FunctionCall functionCall(String name, Window window, boolean distinct, String... args) { - return aliases - .stream() - .map(arg -> arg.toSymbol(symbolAliases).toSymbolReference()) - .collect(toImmutableList()); + return new FunctionCall(QualifiedName.of(name), Optional.of(window), distinct, toExpressionList(args)); } - private static List toSymbolAliases(List aliases) + public static FunctionCall functionCall(String name, String... args) { - return aliases - .stream() - .map(PlanMatchPattern::symbol) - .collect(toImmutableList()); - } - - public static ExpectedValueProvider specification( - List partitionBy, - List orderBy, - Map orderings) - { - return new SpecificationProvider( - partitionBy - .stream() - .map(SymbolAlias::new) - .collect(toImmutableList()), - orderBy - .stream() - .map(SymbolAlias::new) - .collect(toImmutableList()), - orderings - .entrySet() - .stream() - .collect(toImmutableMap(entry -> new SymbolAlias(entry.getKey()), Map.Entry::getValue))); + return new RelaxedEqualityFunctionCall(QualifiedName.of(name), toExpressionList(args)); } @Override @@ -453,8 +256,66 @@ private void toString(StringBuilder builder, int indent) } } - private static String indentString(int indent) + private String indentString(int indent) { return Strings.repeat(" ", indent); } + + private static class AnySymbolReference + extends SymbolReference + { + AnySymbolReference() + { + super("*"); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + + if (o == null || !SymbolReference.class.isInstance(o)) { + return false; + } + + return true; + } + + @Override + public int hashCode() + { + throw new UnsupportedOperationException(); + } + } + + private static class RelaxedEqualityFunctionCall + extends FunctionCall + { + RelaxedEqualityFunctionCall(QualifiedName name, List arguments) + { + super(name, arguments); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || obj.getClass() != FunctionCall.class) { + return false; + } + FunctionCall o = (FunctionCall) obj; + return Objects.equals(getName(), o.getName()) && + Objects.equals(getArguments(), o.getArguments()); + } + + @Override + public int hashCode() + { + return Objects.hash(getName(), getArguments()); + } + } } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/SymbolAlias.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanMatchingContext.java similarity index 50% rename from presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/SymbolAlias.java rename to presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanMatchingContext.java index 561c5ca31950..b1405944165a 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/SymbolAlias.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanMatchingContext.java @@ -13,28 +13,33 @@ */ package com.facebook.presto.sql.planner.assertions; -import com.facebook.presto.sql.planner.Symbol; - import static java.util.Objects.requireNonNull; -class SymbolAlias - implements PlanTestSymbol +final class PlanMatchingContext { - private final String alias; + private final ExpressionAliases expressionAliases; + private final PlanMatchPattern pattern; + + PlanMatchingContext(PlanMatchPattern pattern) + { + this(new ExpressionAliases(), pattern); + } - SymbolAlias(String alias) + PlanMatchingContext(ExpressionAliases expressionAliases, PlanMatchPattern pattern) { - this.alias = requireNonNull(alias, "alias is null"); + requireNonNull(expressionAliases, "expressionAliases is null"); + requireNonNull(pattern, "pattern is null"); + this.expressionAliases = new ExpressionAliases(expressionAliases); + this.pattern = pattern; } - public Symbol toSymbol(SymbolAliases aliases) + PlanMatchPattern getPattern() { - return Symbol.from(aliases.get(alias)); + return pattern; } - @Override - public String toString() + ExpressionAliases getExpressionAliases() { - return alias; + return expressionAliases; } } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanMatchingState.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanMatchingState.java index 13c7430b6261..1429551f993d 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanMatchingState.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanMatchingState.java @@ -17,26 +17,19 @@ import java.util.List; +import static com.google.common.base.Preconditions.checkArgument; import static java.util.Objects.requireNonNull; -/** - * A PlanMatchingState is a list of patterns representing a potential match - * for the sources of a PlanMatchNode. There is a 1:1 correspondence between - * the elements of the patterns and the sources. - * - * Example: - * During the matching process, a PlanMatchingState with patterns [pattern1, pattern2] - * is generated while visiting a PlanNode with sources [sourceA, sourceB]. - * The matching process will attempt the following matches: - * [pattern1 : sourceA, pattern2 : sourceB] - */ final class PlanMatchingState { private final List patterns; + private final ExpressionAliases expressionAliases; - PlanMatchingState(List patterns) + PlanMatchingState(List patterns, ExpressionAliases expressionAliases) { + requireNonNull(expressionAliases, "expressionAliases is null"); requireNonNull(patterns, "matchers is null"); + this.expressionAliases = new ExpressionAliases(expressionAliases); this.patterns = ImmutableList.copyOf(patterns); } @@ -45,6 +38,12 @@ boolean isTerminated() return patterns.isEmpty() || patterns.stream().allMatch(PlanMatchPattern::isTerminated); } + PlanMatchingContext createContext(int matcherId) + { + checkArgument(matcherId < patterns.size(), "mactcherId out of scope"); + return new PlanMatchingContext(expressionAliases, patterns.get(matcherId)); + } + List getPatterns() { return patterns; diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanMatchingVisitor.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanMatchingVisitor.java index 35891ba17e25..9c16928b4ef0 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanMatchingVisitor.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanMatchingVisitor.java @@ -15,24 +15,18 @@ import com.facebook.presto.Session; import com.facebook.presto.metadata.Metadata; -import com.facebook.presto.sql.planner.Symbol; -import com.facebook.presto.sql.planner.plan.ExchangeNode; import com.facebook.presto.sql.planner.plan.PlanNode; import com.facebook.presto.sql.planner.plan.PlanVisitor; import com.facebook.presto.sql.planner.plan.ProjectNode; -import com.facebook.presto.sql.tree.Expression; -import com.google.common.collect.ImmutableMap; import java.util.List; -import static com.facebook.presto.sql.planner.assertions.MatchResult.NO_MATCH; -import static com.facebook.presto.sql.planner.assertions.MatchResult.match; +import static com.facebook.presto.util.ImmutableCollectors.toImmutableList; import static com.google.common.base.Preconditions.checkState; -import static java.lang.String.format; import static java.util.Objects.requireNonNull; final class PlanMatchingVisitor - extends PlanVisitor + extends PlanVisitor { private final Metadata metadata; private final Session session; @@ -44,137 +38,43 @@ final class PlanMatchingVisitor } @Override - public MatchResult visitExchange(ExchangeNode node, PlanMatchPattern pattern) + public Boolean visitProject(ProjectNode node, PlanMatchingContext context) { - checkState(node.getType() == ExchangeNode.Type.GATHER, "Only GATHER is supported"); - List> allInputs = node.getInputs(); - checkState(allInputs.size() == 1, "Multiple lists of inputs are not supported yet"); - - List inputs = allInputs.get(0); - List outputs = node.getOutputSymbols(); - - MatchResult result = super.visitExchange(node, pattern); - - if (!result.isMatch()) { - return result; - } - - ImmutableMap.Builder assignments = ImmutableMap.builder(); - for (int i = 0; i < inputs.size(); ++i) { - assignments.put(outputs.get(i), inputs.get(i).toSymbolReference()); - } - - return match(result.getAliases().updateAssignments(assignments.build())); - } - - @Override - public MatchResult visitProject(ProjectNode node, PlanMatchPattern pattern) - { - MatchResult result = super.visitProject(node, pattern); - - if (!result.isMatch()) { - return result; - } - - return match(result.getAliases().replaceAssignments(node.getAssignments())); + context.getExpressionAliases().updateAssignments(node.getAssignments()); + return super.visitProject(node, context); } @Override - protected MatchResult visitPlan(PlanNode node, PlanMatchPattern pattern) + protected Boolean visitPlan(PlanNode node, PlanMatchingContext context) { - List states = pattern.shapeMatches(node); + List states = context.getPattern().matches(node, session, metadata, context.getExpressionAliases()); - // No shape match; don't need to check the internals of any of the nodes. if (states.isEmpty()) { - return NO_MATCH; + return false; } - // Leaf node in the plan. if (node.getSources().isEmpty()) { - return matchLeaf(node, pattern, states); + return !filterTerminated(states).isEmpty(); } - MatchResult result = NO_MATCH; for (PlanMatchingState state : states) { - // Traverse down the tree, checking to see if the sources match the source patterns in state. - MatchResult sourcesMatch = matchSources(node, state); - - if (!sourcesMatch.isMatch()) { - continue; + checkState(node.getSources().size() == state.getPatterns().size(), "Matchers count does not match count of sources"); + int i = 0; + boolean sourcesMatch = true; + for (PlanNode source : node.getSources()) { + sourcesMatch = sourcesMatch && source.accept(this, state.createContext(i++)); } - - // Try upMatching this node with the the aliases gathered from the source nodes. - SymbolAliases allSourceAliases = sourcesMatch.getAliases(); - MatchResult matchResult = pattern.detailMatches(node, session, metadata, allSourceAliases); - if (matchResult.isMatch()) { - checkState(result == NO_MATCH, format("Ambiguous match on node %s", node)); - result = match(allSourceAliases.withNewAliases(matchResult.getAliases())); + if (sourcesMatch) { + return true; } } - return result; + return false; } - private MatchResult matchLeaf(PlanNode node, PlanMatchPattern pattern, List states) + private List filterTerminated(List states) { - MatchResult result = NO_MATCH; - - for (PlanMatchingState state : states) { - // Don't consider un-terminated PlanMatchingStates. - if (!state.isTerminated()) { - continue; - } - - /* - * We have to call detailMatches for two reasons: - * 1) Make sure there aren't any mismatches checking the internals of a leaf node. - * 2) Collect the aliases from the source nodes so we can add them to - * SymbolAliases. They'll be needed further up. - */ - MatchResult matchResult = pattern.detailMatches(node, session, metadata, new SymbolAliases()); - if (matchResult.isMatch()) { - checkState(result == NO_MATCH, format("Ambiguous match on leaf node %s", node)); - result = matchResult; - } - } - - return result; - } - - /* - * This is a little counter-intuitive. Calling matchSources calls - * source.accept, which (eventually) ends up calling into visitPlan - * recursively. Assuming the plan and pattern currently being matched - * actually match each other, eventually you hit the leaf nodes. At that - * point, visitPlan starts by returning the match result for the leaf nodes - * containing the symbol aliases needed by further up. - * - * For the non-leaf nodes, an invocation of matchSources returns a match - * result for a successful match containing the union of all of the symbol - * aliases added by the sources of the node currently being visited. - * - * Visiting that node proceeds by trying to apply the current pattern's - * detailMatches() method to the node being visited. When a match is found, - * visitPlan returns a match result containing the aliases for all of the - * current node's sources, and the aliases for the current node. - */ - private MatchResult matchSources(PlanNode node, PlanMatchingState state) - { - List sourcePatterns = state.getPatterns(); - checkState(node.getSources().size() == sourcePatterns.size(), "Matchers count does not match count of sources"); - - int i = 0; - SymbolAliases.Builder allSourceAliases = SymbolAliases.builder(); - for (PlanNode source : node.getSources()) { - // Match sources to patterns 1:1 - MatchResult matchResult = source.accept(this, sourcePatterns.get(i++)); - if (!matchResult.isMatch()) { - return NO_MATCH; - } - - // Add the per-source aliases to the per-state aliases. - allSourceAliases.putAll(matchResult.getAliases()); - } - - return match(allSourceAliases.build()); + return states.stream() + .filter(PlanMatchingState::isTerminated) + .collect(toImmutableList()); } } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanNodeMatcher.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanNodeMatcher.java index d755cc8f88af..4ed71b16e644 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanNodeMatcher.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanNodeMatcher.java @@ -17,9 +17,7 @@ import com.facebook.presto.metadata.Metadata; import com.facebook.presto.sql.planner.plan.PlanNode; -import static com.facebook.presto.sql.planner.assertions.MatchResult.match; import static com.google.common.base.MoreObjects.toStringHelper; -import static com.google.common.base.Preconditions.checkState; import static java.util.Objects.requireNonNull; final class PlanNodeMatcher @@ -33,18 +31,11 @@ public PlanNodeMatcher(Class nodeClass) } @Override - public boolean shapeMatches(PlanNode node) + public boolean matches(PlanNode node, Session session, Metadata metadata, ExpressionAliases expressionAliases) { return node.getClass().equals(nodeClass); } - @Override - public MatchResult detailMatches(PlanNode node, Session session, Metadata metadata, SymbolAliases symbolAliases) - { - checkState(shapeMatches(node), "Plan testing framework error: shapeMatches returned false in detailMatches in %s", this.getClass().getName()); - return match(); - } - @Override public String toString() { diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanTestSymbol.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanTestSymbol.java deleted file mode 100644 index 25aff45ee3d5..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/PlanTestSymbol.java +++ /dev/null @@ -1,21 +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.sql.planner.assertions; - -import com.facebook.presto.sql.planner.Symbol; - -public interface PlanTestSymbol -{ - Symbol toSymbol(SymbolAliases aliases); -} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/RvalueMatcher.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/RvalueMatcher.java deleted file mode 100644 index 84a74c303aac..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/RvalueMatcher.java +++ /dev/null @@ -1,36 +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.sql.planner.assertions; - -import com.facebook.presto.Session; -import com.facebook.presto.metadata.Metadata; -import com.facebook.presto.sql.planner.Symbol; -import com.facebook.presto.sql.planner.plan.PlanNode; - -import java.util.Optional; - -public interface RvalueMatcher -{ - /** - * Get the unique symbol that is assigned an rvalue matched by the RvalueMatcher in node. - * RvalueMatchers can match anything that can be assigned to a Symbol, and may be specialized - * based on the type of the node or the type of the value that is being assigned. For example, - * TableScanNodes assign ColumnHandles to Symbols and AggregationNodes assign FunctionCalls - * to Symbols. - * - * The assigned symbol is identified by matching the value on the right side of the assignment; - * the rvalue. If no match is found in the node, getAssignedSymbol must return Optional.empty(). - */ - Optional getAssignedSymbol(PlanNode node, Session session, Metadata metadata, SymbolAliases symbolAliases); -} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/SemiJoinMatcher.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/SemiJoinMatcher.java index 4e5d7b5ce48c..ca1eecf17261 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/SemiJoinMatcher.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/SemiJoinMatcher.java @@ -18,10 +18,7 @@ import com.facebook.presto.sql.planner.plan.PlanNode; import com.facebook.presto.sql.planner.plan.SemiJoinNode; -import static com.facebook.presto.sql.planner.assertions.MatchResult.NO_MATCH; -import static com.facebook.presto.sql.planner.assertions.MatchResult.match; import static com.google.common.base.MoreObjects.toStringHelper; -import static com.google.common.base.Preconditions.checkState; import static java.util.Objects.requireNonNull; final class SemiJoinMatcher @@ -39,23 +36,16 @@ final class SemiJoinMatcher } @Override - public boolean shapeMatches(PlanNode node) + public boolean matches(PlanNode node, Session session, Metadata metadata, ExpressionAliases expressionAliases) { - return node instanceof SemiJoinNode; - } - - @Override - public MatchResult detailMatches(PlanNode node, Session session, Metadata metadata, SymbolAliases symbolAliases) - { - checkState(shapeMatches(node), "Plan testing framework error: shapeMatches returned false in detailMatches in %s", this.getClass().getName()); - - SemiJoinNode semiJoinNode = (SemiJoinNode) node; - if (!(symbolAliases.get(sourceSymbolAlias).equals(semiJoinNode.getSourceJoinSymbol().toSymbolReference()) && - symbolAliases.get(filteringSymbolAlias).equals(semiJoinNode.getFilteringSourceJoinSymbol().toSymbolReference()))) { - return NO_MATCH; + if (node instanceof SemiJoinNode) { + SemiJoinNode semiJoinNode = (SemiJoinNode) node; + expressionAliases.put(sourceSymbolAlias, semiJoinNode.getSourceJoinSymbol().toSymbolReference()); + expressionAliases.put(filteringSymbolAlias, semiJoinNode.getFilteringSourceJoinSymbol().toSymbolReference()); + expressionAliases.put(outputAlias, semiJoinNode.getSemiJoinOutput().toSymbolReference()); + return true; } - - return match(outputAlias, semiJoinNode.getSemiJoinOutput().toSymbolReference()); + return false; } @Override diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/SpecificationProvider.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/SpecificationProvider.java deleted file mode 100644 index a1e5a0ecb604..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/SpecificationProvider.java +++ /dev/null @@ -1,73 +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.sql.planner.assertions; - -import com.facebook.presto.spi.block.SortOrder; -import com.facebook.presto.sql.planner.plan.WindowNode; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; - -import java.util.List; -import java.util.Map; - -import static com.facebook.presto.util.ImmutableCollectors.toImmutableList; -import static com.facebook.presto.util.ImmutableCollectors.toImmutableMap; -import static com.google.common.base.MoreObjects.toStringHelper; -import static java.util.Objects.requireNonNull; - -public class SpecificationProvider - implements ExpectedValueProvider -{ - private final List partitionBy; - private final List orderBy; - private final Map orderings; - - SpecificationProvider( - List partitionBy, - List orderBy, - Map orderings) - { - this.partitionBy = ImmutableList.copyOf(requireNonNull(partitionBy, "partitionBy is null")); - this.orderBy = ImmutableList.copyOf(requireNonNull(orderBy, "orderBy is null")); - this.orderings = ImmutableMap.copyOf(requireNonNull(orderings, "orderings is null")); - } - - @Override - public WindowNode.Specification getExpectedValue(SymbolAliases aliases) - { - return new WindowNode.Specification( - partitionBy - .stream() - .map(alias -> alias.toSymbol(aliases)) - .collect(toImmutableList()), - orderBy - .stream() - .map(alias -> alias.toSymbol(aliases)) - .collect(toImmutableList()), - orderings - .entrySet() - .stream() - .collect(toImmutableMap(entry -> entry.getKey().toSymbol(aliases), Map.Entry::getValue))); - } - - @Override - public String toString() - { - return toStringHelper(this) - .add("partitionBy", this.partitionBy) - .add("orderBy", this.orderBy) - .add("orderings", this.orderings) - .toString(); - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/SymbolAliases.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/SymbolAliases.java deleted file mode 100644 index f105d0373bf7..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/SymbolAliases.java +++ /dev/null @@ -1,232 +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.sql.planner.assertions; - -import com.facebook.presto.sql.planner.Symbol; -import com.facebook.presto.sql.tree.Expression; -import com.facebook.presto.sql.tree.SymbolReference; -import com.google.common.collect.ImmutableMap; - -import java.util.HashMap; -import java.util.Map; - -import static com.google.common.base.Preconditions.checkState; -import static java.lang.String.format; -import static java.util.Objects.requireNonNull; - -public final class SymbolAliases -{ - private final Map map; - - public SymbolAliases() - { - this.map = ImmutableMap.of(); - } - - private SymbolAliases(Map aliases) - { - this.map = ImmutableMap.copyOf(aliases); - } - - public SymbolAliases(SymbolAliases symbolAliases) - { - requireNonNull(symbolAliases, "symbolAliases are null"); - this.map = ImmutableMap.copyOf(symbolAliases.map); - } - - public static Builder builder() - { - return new Builder(); - } - - public SymbolAliases withNewAliases(SymbolAliases sourceAliases) - { - Builder builder = new Builder(this); - - for (Map.Entry alias : sourceAliases.map.entrySet()) { - builder.put(alias.getKey(), alias.getValue()); - } - - return builder.build(); - } - - public SymbolReference get(String alias) - { - alias = toKey(alias); - SymbolReference result = map.get(alias); - /* - * It's still kind of an open question if the right combination of anyTree() and - * a sufficiently complex and/or ambiguous plan might make throwing here a - * theoretically incorrect thing to do. - * - * If you run into a case that you think justifies changing this, please consider - * that it's already pretty hard to determine if a failure is because the test - * is written incorrectly or because the actual plan really doesn't match a - * correctly written test. Having this throw makes it a lot easier to track down - * missing aliases in incorrect plans. - */ - checkState(result != null, format("missing expression for alias %s", alias)); - return result; - } - - private static String toKey(String alias) - { - // Required because the SqlParser lower cases SymbolReferences in the expressions we parse with it. - return alias.toLowerCase(); - } - - private Map getUpdatedAssignments(Map assignments) - { - ImmutableMap.Builder mapUpdate = ImmutableMap.builder(); - for (Map.Entry assignment : assignments.entrySet()) { - for (Map.Entry existingAlias : map.entrySet()) { - if (assignment.getValue().equals(existingAlias.getValue())) { - // Simple symbol rename - mapUpdate.put(existingAlias.getKey(), assignment.getKey().toSymbolReference()); - } - else if (assignment.getKey().toSymbolReference().equals(existingAlias.getValue())) { - /* - * Special case for nodes that can alias symbols in the node's assignment map. - * In this case, we've already added the alias in the map, but we won't include it - * as a simple rename as covered above. Add the existing alias to the result if - * the LHS of the assignment matches the symbol reference of the existing alias. - * - * This comes up when we alias expressions in project nodes for use further up the tree. - * At the beginning for the function, map contains { NEW_ALIAS: SymbolReference("expr_2" } - * and the assignments map contains { expr_2 := }. - */ - mapUpdate.put(existingAlias.getKey(), existingAlias.getValue()); - } - } - } - return mapUpdate.build(); - } - - /* - * Return a new SymbolAliases that contains a map with the original bindings - * updated based on assignments given that assignments is a map of - * newSymbol := oldSymbolReference. - * - * INCLUDE aliases for SymbolReferences that aren't in assignments.values() - * - * Example: - * SymbolAliases = { "ALIAS1": SymbolReference("foo"), "ALIAS2": SymbolReference("bar")} - * updateAssignments({"baz": SymbolReference("foo")}) - * returns a new - * SymbolAliases = { "ALIAS1": SymbolReference("baz"), "ALIAS2": SymbolReference("bar")} - */ - public SymbolAliases updateAssignments(Map assignments) - { - return builder() - .putAll(this) - .putUnchecked(getUpdatedAssignments(assignments)) - .build(); - } - - /* - * Return a new SymbolAliases that contains a map with the original bindings - * updated based on assignments given that assignments is a map of - * newSymbol := oldSymbolReference. - * - * DISCARD aliases for SymbolReferences that aren't in assignments.values() - * - * Example: - * SymbolAliases = { "ALIAS1": SymbolReference("foo"), "ALIAS2": SymbolReference("bar")} - * updateAssignments({"baz": SymbolReference("foo")}) - * returns a new - * SymbolAliases = { "ALIAS1": SymbolReference("baz") } - * - * When you pass through a project node, all of the aliases need to be updated, and - * aliases for symbols that aren't projected need to be removed. - * - * Example in the context of a Plan: - * PlanMatchPattern.tableScan("nation", ImmutableMap.of("NK", "nationkey", "RK", "regionkey") - * applied to - * TableScanNode { col1 := ColumnHandle(nation, nationkey), col2 := ColumnHandle(nation, regionkey) } - * gives SymbolAliases.map - * { "NK": SymbolReference("col1"), "RK": SymbolReference("col2") } - * - * ... Visit some other nodes, one of which presumably consumes col1, and none of which add any new aliases ... - * - * If we then visit a project node - * Project { value3 := col2 } - * SymbolAliases.map should be - * { "RK": SymbolReference("value3") } - */ - public SymbolAliases replaceAssignments(Map assignments) - { - return new SymbolAliases(getUpdatedAssignments(assignments)); - } - - public static class Builder - { - Map bindings; - - private Builder() - { - bindings = new HashMap<>(); - } - - private Builder(SymbolAliases initialAliases) - { - bindings = new HashMap<>(initialAliases.map); - } - - public Builder put(String alias, SymbolReference symbolReference) - { - requireNonNull(alias, "alias is null"); - requireNonNull(symbolReference, "symbolReference is null"); - - alias = toKey(alias); - - // Special case to allow identity binding (i.e. "ALIAS" -> expression("ALIAS")) - if (bindings.containsKey(alias) && bindings.get(alias).equals(symbolReference)) { - return this; - } - - checkState(!bindings.containsKey(alias), "Alias '%s' already bound to expression '%s'. Tried to rebind to '%s'", alias, bindings.get(alias), symbolReference); - checkState(!bindings.values().contains(symbolReference), "Expression '%s' is already bound in %s. Tried to rebind as '%s'.", symbolReference, bindings, alias); - bindings.put(alias, symbolReference); - return this; - } - - public Builder putAll(Map aliases) - { - aliases.entrySet() - .forEach(entry -> put(entry.getKey(), entry.getValue())); - return this; - } - - /* - * This is supplied specifically for updateAssigments, which needs to - * update existing bindings that have already been added. Unless you're - * certain you want this behavior, you don't want it. - */ - private Builder putUnchecked(Map aliases) - { - bindings.putAll(aliases); - return this; - } - - public Builder putAll(SymbolAliases aliases) - { - return putAll(aliases.map); - } - - public SymbolAliases build() - { - return new SymbolAliases(bindings); - } - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/SymbolMatcher.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/SymbolMatcher.java new file mode 100644 index 000000000000..0acf8143330f --- /dev/null +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/SymbolMatcher.java @@ -0,0 +1,63 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.sql.planner.assertions; + +import com.facebook.presto.Session; +import com.facebook.presto.metadata.Metadata; +import com.facebook.presto.sql.planner.Symbol; +import com.facebook.presto.sql.planner.plan.PlanNode; + +import java.util.regex.Pattern; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static com.google.common.base.Preconditions.checkState; + +final class SymbolMatcher + implements Matcher +{ + private final Pattern pattern; + private final String alias; + + SymbolMatcher(String pattern, String alias) + { + this.pattern = Pattern.compile(pattern); + this.alias = alias; + } + + @Override + public boolean matches(PlanNode node, Session session, Metadata metadata, ExpressionAliases expressionAliases) + { + Symbol symbol = null; + for (Symbol outputSymbol : node.getOutputSymbols()) { + if (pattern.matcher(outputSymbol.getName()).find()) { + checkState(symbol == null, "%s symbol was found multiple times in %s", pattern, node.getOutputSymbols()); + symbol = outputSymbol; + } + } + if (symbol != null) { + expressionAliases.put(alias, symbol.toSymbolReference()); + return true; + } + return false; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("alias", alias) + .add("pattern", pattern) + .toString(); + } +} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/TableScanMatcher.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/TableScanMatcher.java index e6f154165764..cf4391c0c466 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/TableScanMatcher.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/TableScanMatcher.java @@ -26,7 +26,6 @@ import java.util.Optional; import static com.google.common.base.MoreObjects.toStringHelper; -import static com.google.common.base.Preconditions.checkState; import static java.util.Objects.requireNonNull; final class TableScanMatcher @@ -48,22 +47,15 @@ public TableScanMatcher(String expectedTableName, Map expectedCo } @Override - public boolean shapeMatches(PlanNode node) + public boolean matches(PlanNode node, Session session, Metadata metadata, ExpressionAliases expressionAliases) { - return node instanceof TableScanNode; - } - - @Override - public MatchResult detailMatches(PlanNode node, Session session, Metadata metadata, SymbolAliases symbolAliases) - { - checkState(shapeMatches(node), "Plan testing framework error: shapeMatches returned false in detailMatches in %s", this.getClass().getName()); - - TableScanNode tableScanNode = (TableScanNode) node; - TableMetadata tableMetadata = metadata.getTableMetadata(session, tableScanNode.getTable()); - String actualTableName = tableMetadata.getTable().getTableName(); - return new MatchResult( - expectedTableName.equalsIgnoreCase(actualTableName) && - domainMatches(tableScanNode, session, metadata)); + if (node instanceof TableScanNode) { + TableScanNode tableScanNode = (TableScanNode) node; + TableMetadata tableMetadata = metadata.getTableMetadata(session, tableScanNode.getTable()); + String actualTableName = tableMetadata.getTable().getTableName(); + return expectedTableName.equalsIgnoreCase(actualTableName) && domainMatches(tableScanNode, session, metadata); + } + return false; } private boolean domainMatches(TableScanNode tableScanNode, Session session, Metadata metadata) diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/TestExpressionVerifier.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/TestExpressionVerifier.java index 24c013382c66..a7c014f42e0e 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/TestExpressionVerifier.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/TestExpressionVerifier.java @@ -15,12 +15,10 @@ import com.facebook.presto.sql.parser.SqlParser; import com.facebook.presto.sql.tree.Expression; -import com.facebook.presto.sql.tree.SymbolReference; import org.testng.annotations.Test; import static com.facebook.presto.sql.ExpressionUtils.rewriteQualifiedNamesToSymbolReferences; import static org.testng.Assert.assertTrue; -import static org.testng.AssertJUnit.assertFalse; public class TestExpressionVerifier { @@ -31,30 +29,11 @@ public void test() { Expression actual = expression("NOT(orderkey = 3 AND custkey = 3 AND orderkey < 10)"); - SymbolAliases symbolAliases = SymbolAliases.builder() - .put("X", new SymbolReference("orderkey")) - .put("Y", new SymbolReference("custkey")) - .build(); - - ExpressionVerifier verifier = new ExpressionVerifier(symbolAliases); + ExpressionVerifier verifier = new ExpressionVerifier(new ExpressionAliases()); assertTrue(verifier.process(actual, expression("NOT(X = 3 AND Y = 3 AND X < 10)"))); assertThrows(() -> verifier.process(actual, expression("NOT(X = 3 AND Y = 3 AND Z < 10)"))); - assertFalse(verifier.process(actual, expression("NOT(X = 3 AND X = 3 AND X < 10)"))); - } - - @Test - public void testCast() - throws Exception - { - SymbolAliases aliases = SymbolAliases.builder() - .put("X", new SymbolReference("orderkey")) - .build(); - - ExpressionVerifier verifier = new ExpressionVerifier(aliases); - assertTrue(verifier.process(expression("CAST('2' AS varchar)"), expression("CAST('2' AS varchar)"))); - assertFalse(verifier.process(expression("CAST('2' AS varchar)"), expression("CAST('2' AS bigint)"))); - assertTrue(verifier.process(expression("CAST(orderkey AS varchar)"), expression("CAST(X AS varchar)"))); + assertThrows(() -> verifier.process(actual, expression("NOT(X = 3 AND X = 3 AND X < 10)"))); } private Expression expression(String sql) @@ -66,7 +45,7 @@ private static void assertThrows(Runnable runnable) { try { runnable.run(); - throw new AssertionError("Method didn't throw exception as expected"); + throw new AssertionError("Method din't throw an exception as it was expected"); } catch (Exception expected) { } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ValuesMatcher.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ValuesMatcher.java deleted file mode 100644 index fa4768a49cba..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/ValuesMatcher.java +++ /dev/null @@ -1,45 +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.sql.planner.assertions; - -import com.facebook.presto.Session; -import com.facebook.presto.metadata.Metadata; -import com.facebook.presto.sql.planner.Symbol; -import com.facebook.presto.sql.planner.plan.PlanNode; -import com.facebook.presto.sql.planner.plan.ValuesNode; - -import java.util.Optional; - -public class ValuesMatcher - implements RvalueMatcher -{ - int outputIndex; - - public ValuesMatcher(int outputIndex) - { - this.outputIndex = outputIndex; - } - - @Override - public Optional getAssignedSymbol(PlanNode node, Session session, Metadata metadata, SymbolAliases symbolAliases) - { - if (!(node instanceof ValuesNode)) { - return Optional.empty(); - } - - ValuesNode valuesNode = (ValuesNode) node; - - return Optional.of(valuesNode.getOutputSymbols().get(outputIndex)); - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/WindowFunctionMatcher.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/WindowFunctionMatcher.java deleted file mode 100644 index 6c39ceb5f274..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/WindowFunctionMatcher.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.facebook.presto.sql.planner.assertions; - -import com.facebook.presto.Session; -import com.facebook.presto.metadata.Metadata; -import com.facebook.presto.sql.planner.Symbol; -import com.facebook.presto.sql.planner.plan.PlanNode; -import com.facebook.presto.sql.planner.plan.WindowNode; -import com.facebook.presto.sql.tree.FunctionCall; - -import java.util.Map; -import java.util.Optional; - -import static com.google.common.base.Preconditions.checkState; -import static java.util.Objects.requireNonNull; - -public class WindowFunctionMatcher - implements RvalueMatcher -{ - private final ExpectedValueProvider callMaker; - - public WindowFunctionMatcher(ExpectedValueProvider callMaker) - { - this.callMaker = requireNonNull(callMaker, "functionCall is null"); - } - - @Override - public Optional getAssignedSymbol(PlanNode node, Session session, Metadata metadata, SymbolAliases symbolAliases) - { - Optional result = Optional.empty(); - if (!(node instanceof WindowNode)) { - return result; - } - - WindowNode windowNode = (WindowNode) node; - - FunctionCall expectedCall = callMaker.getExpectedValue(symbolAliases); - for (Map.Entry assignment : windowNode.getWindowFunctions().entrySet()) { - if (expectedCall.equals(assignment.getValue().getFunctionCall())) { - checkState(!result.isPresent(), "Ambiguous function calls in %s", windowNode); - result = Optional.of(assignment.getKey()); - } - } - - return result; - } - - @Override - public String toString() - { - return callMaker.toString(); - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/WindowMatcher.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/WindowMatcher.java index 538349e38789..9349937daa48 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/WindowMatcher.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/assertions/WindowMatcher.java @@ -18,47 +18,58 @@ import com.facebook.presto.metadata.Metadata; import com.facebook.presto.sql.planner.plan.PlanNode; import com.facebook.presto.sql.planner.plan.WindowNode; +import com.facebook.presto.sql.tree.FunctionCall; +import com.google.common.collect.ImmutableList; + +import java.util.LinkedList; +import java.util.List; +import java.util.stream.Collectors; import static com.google.common.base.MoreObjects.toStringHelper; -import static com.google.common.base.Preconditions.checkState; +import static java.util.Objects.requireNonNull; final class WindowMatcher implements Matcher { - private final ExpectedValueProvider specification; + private final List functionCalls; - WindowMatcher( - ExpectedValueProvider specification) + WindowMatcher(List functionCalls) { - this.specification = specification; + this.functionCalls = ImmutableList.copyOf(requireNonNull(functionCalls, "functionCalls is null")); } @Override - public boolean shapeMatches(PlanNode node) + public boolean matches(PlanNode node, Session session, Metadata metadata, ExpressionAliases expressionAliases) { - return node instanceof WindowNode; - } - - @Override - public MatchResult detailMatches(PlanNode node, Session session, Metadata metadata, SymbolAliases symbolAliases) - { - checkState(shapeMatches(node), "Plan testing framework error: shapeMatches returned false in detailMatches in %s", this.getClass().getName()); + if (!(node instanceof WindowNode)) { + return false; + } WindowNode windowNode = (WindowNode) node; + LinkedList actualCalls = windowNode.getWindowFunctions().values().stream() + .map(WindowNode.Function::getFunctionCall) + .collect(Collectors.toCollection(LinkedList::new)); + + if (actualCalls.size() != functionCalls.size()) { + return false; + } + + for (FunctionCall expectedCall : functionCalls) { + if (!actualCalls.remove(expectedCall)) { + // Found an expectedCall not in expectedCalls. + return false; + } + } - /* - * Window functions produce a symbol (the result of the function call) that we might - * want to bind to an alias so we can reference it further up the tree. As such, - * they need to be matched with an Alias matcher so we can bind the symbol if desired. - */ - return new MatchResult(windowNode.getSpecification().equals(specification.getExpectedValue(symbolAliases))); + // expectedCalls was missing something in actualCalls. + return actualCalls.isEmpty(); } @Override public String toString() { return toStringHelper(this) - .add("specification", specification) + .add("functionCalls", functionCalls) .toString(); } } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestMergeWindows.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestMergeWindows.java index b70cf03e328a..749d7d3122a3 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestMergeWindows.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestMergeWindows.java @@ -13,17 +13,19 @@ */ package com.facebook.presto.sql.planner.optimizations; -import com.facebook.presto.spi.block.SortOrder; +import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.sql.planner.Plan; -import com.facebook.presto.sql.planner.assertions.BasePlanTest; -import com.facebook.presto.sql.planner.assertions.ExpectedValueProvider; import com.facebook.presto.sql.planner.assertions.PlanAssert; import com.facebook.presto.sql.planner.assertions.PlanMatchPattern; import com.facebook.presto.sql.planner.plan.JoinNode; import com.facebook.presto.sql.planner.plan.WindowNode; import com.facebook.presto.sql.tree.FrameBound; +import com.facebook.presto.sql.tree.SortItem; +import com.facebook.presto.sql.tree.SymbolReference; +import com.facebook.presto.sql.tree.Window; import com.facebook.presto.sql.tree.WindowFrame; import com.facebook.presto.testing.LocalQueryRunner; +import com.facebook.presto.tpch.TpchConnectorFactory; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.intellij.lang.annotations.Language; @@ -35,71 +37,44 @@ import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.any; import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.anyNot; import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.anyTree; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.expression; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.filter; import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.functionCall; import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.join; import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.project; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.specification; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.tableScan; import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.window; +import static com.facebook.presto.testing.TestingSession.testSessionBuilder; public class TestMergeWindows - extends BasePlanTest { - private static final String SUPPKEY_ALIAS = "SUPPKEY"; - private static final String ORDERKEY_ALIAS = "ORDERKEY"; - private static final String SHIPDATE_ALIAS = "SHIPDATE"; - private static final String QUANTITY_ALIAS = "QUANTITY"; - private static final String DISCOUNT_ALIAS = "DISCOUNT"; - private static final String EXTENDEDPRICE_ALIAS = "EXTENDEDPRICE"; - - private static final PlanMatchPattern LINEITEM_TABLESCAN_DOQSS = tableScan( - "lineitem", - ImmutableMap.of(QUANTITY_ALIAS, "quantity", - DISCOUNT_ALIAS, "discount", - SUPPKEY_ALIAS, "suppkey", - ORDERKEY_ALIAS, "orderkey", - SHIPDATE_ALIAS, "shipdate")); - - private static final PlanMatchPattern LINEITEM_TABLESCAN_DOQS = tableScan( - "lineitem", - ImmutableMap.of(QUANTITY_ALIAS, "quantity", - DISCOUNT_ALIAS, "discount", - SUPPKEY_ALIAS, "suppkey", - ORDERKEY_ALIAS, "orderkey")); - - private static final PlanMatchPattern LINEITEM_TABLESCAN_DEOQS = tableScan( - "lineitem", - ImmutableMap.of(QUANTITY_ALIAS, "quantity", - SUPPKEY_ALIAS, "suppkey", - ORDERKEY_ALIAS, "orderkey", - DISCOUNT_ALIAS, "discount", - EXTENDEDPRICE_ALIAS, "extendedprice")); - - private static final Optional COMMON_FRAME = Optional.of(new WindowFrame( - WindowFrame.Type.ROWS, - new FrameBound(FrameBound.Type.UNBOUNDED_PRECEDING), - Optional.of(new FrameBound(FrameBound.Type.CURRENT_ROW)))); - - private static final Optional UNSPECIFIED_FRAME = Optional.empty(); - - private final ExpectedValueProvider specificationA; - private final ExpectedValueProvider specificationB; + private final LocalQueryRunner queryRunner; + private final WindowFrame commonFrame; + private final Window windowA; + private final Window windowB; public TestMergeWindows() { - super(); + this.queryRunner = new LocalQueryRunner(testSessionBuilder() + .setCatalog("local") + .setSchema("tiny") + .build()); - specificationA = specification( - ImmutableList.of(SUPPKEY_ALIAS), - ImmutableList.of(ORDERKEY_ALIAS), - ImmutableMap.of(ORDERKEY_ALIAS, SortOrder.ASC_NULLS_LAST)); + queryRunner.createCatalog(queryRunner.getDefaultSession().getCatalog().get(), + new TpchConnectorFactory(1), + ImmutableMap.of()); + + commonFrame = new WindowFrame( + WindowFrame.Type.ROWS, + new FrameBound(FrameBound.Type.UNBOUNDED_PRECEDING), + Optional.of(new FrameBound(FrameBound.Type.CURRENT_ROW))); - specificationB = specification( - ImmutableList.of(ORDERKEY_ALIAS), - ImmutableList.of(SHIPDATE_ALIAS), - ImmutableMap.of(SHIPDATE_ALIAS, SortOrder.ASC_NULLS_LAST)); + windowA = new Window( + ImmutableList.of(new SymbolReference("suppkey")), + ImmutableList.of(new SortItem(new SymbolReference("orderkey"), SortItem.Ordering.ASCENDING, SortItem.NullOrdering.UNDEFINED)), + Optional.of(commonFrame)); + + windowB = new Window( + ImmutableList.of(new SymbolReference("orderkey")), + ImmutableList.of(new SortItem(new SymbolReference("shipdate"), SortItem.Ordering.ASCENDING, SortItem.NullOrdering.UNDEFINED)), + Optional.of(commonFrame)); } /** @@ -139,18 +114,20 @@ public void testMergeableWindowsAllOptimizers() PlanMatchPattern pattern = anyTree( - window(specificationB, - ImmutableList.of( - functionCall("sum", COMMON_FRAME, ImmutableList.of(QUANTITY_ALIAS))), + window(ImmutableList.of( + functionCall("sum", windowB, false, "quantity")), anyTree( - window(specificationA, - ImmutableList.of( - functionCall("sum", COMMON_FRAME, ImmutableList.of(QUANTITY_ALIAS)), - functionCall("sum", COMMON_FRAME, ImmutableList.of(DISCOUNT_ALIAS))), + window(ImmutableList.of( + functionCall("sum", windowA, false, "quantity"), + functionCall("sum", windowA, false, "discount")), anyNot(WindowNode.class, - anyTree(LINEITEM_TABLESCAN_DOQSS)))))); + anyTree()))))); - assertPlan(sql, pattern); + Plan actualPlan = queryRunner.inTransaction(transactionSession -> queryRunner.createPlan(transactionSession, sql)); + queryRunner.inTransaction(transactionSession -> { + PlanAssert.assertPlan(transactionSession, queryRunner.getMetadata(), actualPlan, pattern); + return null; + }); } @Test @@ -164,14 +141,12 @@ public void testIdenticalWindowSpecificationsABA() assertUnitPlan(sql, anyTree( - window(specificationB, - ImmutableList.of( - functionCall("sum", COMMON_FRAME, ImmutableList.of(QUANTITY_ALIAS))), - window(specificationA, - ImmutableList.of( - functionCall("sum", COMMON_FRAME, ImmutableList.of(QUANTITY_ALIAS)), - functionCall("sum", COMMON_FRAME, ImmutableList.of(DISCOUNT_ALIAS))), - LINEITEM_TABLESCAN_DOQSS)))); + window(ImmutableList.of( + functionCall("sum", windowB, false, "quantity")), + window(ImmutableList.of( + functionCall("sum", windowA, false, "quantity"), + functionCall("sum", windowA, false, "discount")), + anyNot(WindowNode.class))))); } @Test @@ -185,15 +160,12 @@ public void testIdenticalWindowSpecificationsABcpA() assertUnitPlan(sql, anyTree( - window(specificationA, - ImmutableList.of(functionCall("sum", COMMON_FRAME, ImmutableList.of(DISCOUNT_ALIAS))), - window(specificationB, - ImmutableList.of(functionCall("lag", COMMON_FRAME, ImmutableList.of(QUANTITY_ALIAS, "ONE", "ZERO"))), - project(ImmutableMap.of("ONE", expression("CAST(1 AS bigint)"), "ZERO", expression("0.0")), - window(specificationA, - ImmutableList.of( - functionCall("sum", COMMON_FRAME, ImmutableList.of(QUANTITY_ALIAS))), - LINEITEM_TABLESCAN_DOQSS)))))); + window(ImmutableList.of(functionCall("sum", windowA, false, "discount")), + window(ImmutableList.of(functionCall("lag", windowB, false, "quantity", "*", "*")), + project( + window(ImmutableList.of( + functionCall("sum", windowA, false, "quantity")), + any())))))); } @Test @@ -207,29 +179,27 @@ public void testIdenticalWindowSpecificationsAAcpA() assertUnitPlan(sql, anyTree( - window(specificationA, - ImmutableList.of( - functionCall("sum", COMMON_FRAME, ImmutableList.of(DISCOUNT_ALIAS)), - functionCall("lag", COMMON_FRAME, ImmutableList.of(QUANTITY_ALIAS, "ONE", "ZERO"))), - project(ImmutableMap.of("ONE", expression("CAST(1 AS bigint)"), "ZERO", expression("0.0")), - window(specificationA, - ImmutableList.of( - functionCall("sum", COMMON_FRAME, ImmutableList.of(QUANTITY_ALIAS))), - LINEITEM_TABLESCAN_DOQS))))); + window(ImmutableList.of( + functionCall("sum", windowA, false, "discount"), + functionCall("lag", windowA, false, "quantity", "*", "*")), + project( + window(ImmutableList.of( + functionCall("sum", windowA, false, "quantity")), + any()))))); } @Test public void testIdenticalWindowSpecificationsDefaultFrame() { - ExpectedValueProvider specificationC = specification( - ImmutableList.of(SUPPKEY_ALIAS), - ImmutableList.of(ORDERKEY_ALIAS), - ImmutableMap.of(ORDERKEY_ALIAS, SortOrder.ASC_NULLS_LAST)); + Window windowC = new Window( + ImmutableList.of(new SymbolReference("suppkey")), + ImmutableList.of(new SortItem(new SymbolReference("orderkey"), SortItem.Ordering.ASCENDING, SortItem.NullOrdering.UNDEFINED)), + Optional.empty()); - ExpectedValueProvider specificationD = specification( - ImmutableList.of(ORDERKEY_ALIAS), - ImmutableList.of(SHIPDATE_ALIAS), - ImmutableMap.of(SHIPDATE_ALIAS, SortOrder.ASC_NULLS_LAST)); + Window windowD = new Window( + ImmutableList.of(new SymbolReference("orderkey")), + ImmutableList.of(new SortItem(new SymbolReference("shipdate"), SortItem.Ordering.ASCENDING, SortItem.NullOrdering.UNDEFINED)), + Optional.empty()); @Language("SQL") String sql = "SELECT " + "SUM(quantity) OVER (PARTITION By suppkey ORDER BY orderkey), " + @@ -239,33 +209,36 @@ public void testIdenticalWindowSpecificationsDefaultFrame() assertUnitPlan(sql, anyTree( - window(specificationD, - ImmutableList.of( - functionCall("sum", UNSPECIFIED_FRAME, ImmutableList.of(QUANTITY_ALIAS))), - window(specificationC, - ImmutableList.of( - functionCall("sum", UNSPECIFIED_FRAME, ImmutableList.of(QUANTITY_ALIAS)), - functionCall("sum", UNSPECIFIED_FRAME, ImmutableList.of(DISCOUNT_ALIAS))), - LINEITEM_TABLESCAN_DOQSS)))); + window(ImmutableList.of( + functionCall("sum", windowD, false, "quantity")), + window(ImmutableList.of( + functionCall("sum", windowC, false, "quantity"), + functionCall("sum", windowC, false, "discount")), + anyNot(WindowNode.class))))); } @Test public void testMergeDifferentFrames() { - Optional frameC = Optional.of(new WindowFrame( + WindowFrame frameC = new WindowFrame( WindowFrame.Type.ROWS, new FrameBound(FrameBound.Type.UNBOUNDED_PRECEDING), - Optional.of(new FrameBound(FrameBound.Type.CURRENT_ROW)))); + Optional.of(new FrameBound(FrameBound.Type.CURRENT_ROW))); - ExpectedValueProvider specificationC = specification( - ImmutableList.of(SUPPKEY_ALIAS), - ImmutableList.of(ORDERKEY_ALIAS), - ImmutableMap.of(ORDERKEY_ALIAS, SortOrder.ASC_NULLS_LAST)); + Window windowC = new Window( + ImmutableList.of(new SymbolReference("suppkey")), + ImmutableList.of(new SortItem(new SymbolReference("orderkey"), SortItem.Ordering.ASCENDING, SortItem.NullOrdering.UNDEFINED)), + Optional.of(frameC)); - Optional frameD = Optional.of(new WindowFrame( + WindowFrame frameD = new WindowFrame( WindowFrame.Type.ROWS, new FrameBound(FrameBound.Type.CURRENT_ROW), - Optional.of(new FrameBound(FrameBound.Type.UNBOUNDED_FOLLOWING)))); + Optional.of(new FrameBound(FrameBound.Type.UNBOUNDED_FOLLOWING))); + + Window windowD = new Window( + ImmutableList.of(new SymbolReference("suppkey")), + ImmutableList.of(new SortItem(new SymbolReference("orderkey"), SortItem.Ordering.ASCENDING, SortItem.NullOrdering.UNDEFINED)), + Optional.of(frameD)); @Language("SQL") String sql = "SELECT " + "SUM(quantity) OVER (PARTITION BY suppkey ORDER BY orderkey ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) sum_quantity_C, " + @@ -275,26 +248,30 @@ public void testMergeDifferentFrames() assertUnitPlan(sql, anyTree( - window(specificationC, - ImmutableList.of( - functionCall("avg", frameD, ImmutableList.of(QUANTITY_ALIAS)), - functionCall("sum", frameC, ImmutableList.of(DISCOUNT_ALIAS)), - functionCall("sum", frameC, ImmutableList.of(QUANTITY_ALIAS))), - LINEITEM_TABLESCAN_DOQS))); + window(ImmutableList.of( + functionCall("avg", windowD, false, "quantity"), + functionCall("sum", windowC, false, "discount"), + functionCall("sum", windowC, false, "quantity")), + any()))); } @Test public void testMergeDifferentFramesWithDefault() { - Optional frameD = Optional.of(new WindowFrame( + Window windowC = new Window( + ImmutableList.of(new SymbolReference("suppkey")), + ImmutableList.of(new SortItem(new SymbolReference("orderkey"), SortItem.Ordering.ASCENDING, SortItem.NullOrdering.UNDEFINED)), + Optional.empty()); + + WindowFrame frameD = new WindowFrame( WindowFrame.Type.ROWS, new FrameBound(FrameBound.Type.CURRENT_ROW), - Optional.of(new FrameBound(FrameBound.Type.UNBOUNDED_FOLLOWING)))); + Optional.of(new FrameBound(FrameBound.Type.UNBOUNDED_FOLLOWING))); - ExpectedValueProvider specificationD = specification( - ImmutableList.of(SUPPKEY_ALIAS), - ImmutableList.of(ORDERKEY_ALIAS), - ImmutableMap.of(ORDERKEY_ALIAS, SortOrder.ASC_NULLS_LAST)); + Window windowD = new Window( + ImmutableList.of(new SymbolReference("suppkey")), + ImmutableList.of(new SortItem(new SymbolReference("orderkey"), SortItem.Ordering.ASCENDING, SortItem.NullOrdering.UNDEFINED)), + Optional.of(frameD)); @Language("SQL") String sql = "SELECT " + "SUM(quantity) OVER (PARTITION BY suppkey ORDER BY orderkey) sum_quantity_C, " + @@ -304,121 +281,83 @@ public void testMergeDifferentFramesWithDefault() assertUnitPlan(sql, anyTree( - window(specificationD, - ImmutableList.of( - functionCall("avg", frameD, ImmutableList.of(QUANTITY_ALIAS)), - functionCall("sum", UNSPECIFIED_FRAME, ImmutableList.of(DISCOUNT_ALIAS)), - functionCall("sum", UNSPECIFIED_FRAME, ImmutableList.of(QUANTITY_ALIAS))), - LINEITEM_TABLESCAN_DOQS))); + window(ImmutableList.of( + functionCall("avg", windowD, false, "quantity"), + functionCall("sum", windowC, false, "discount"), + functionCall("sum", windowC, false, "quantity")), + any()))); } @Test public void testNotMergeAcrossJoinBranches() { - String rOrderkeyAlias = "R_ORDERKEY"; - String rShipdateAlias = "R_SHIPDATE"; - String rQuantityAlias = "R_QUANTITY"; - @Language("SQL") String sql = "WITH foo AS (" + "SELECT " + "suppkey, orderkey, partkey, " + - "SUM(discount) OVER (PARTITION BY orderkey ORDER BY shipdate, quantity DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) a " + + "SUM(discount) OVER (PARTITION BY orderkey ORDER BY shipdate ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) a " + "FROM lineitem WHERE (partkey = 272 OR partkey = 273) AND suppkey > 50 " + "), " + "bar AS ( " + "SELECT " + "suppkey, orderkey, partkey, " + - "AVG(quantity) OVER (PARTITION BY orderkey ORDER BY shipdate, quantity DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) b " + + "AVG(quantity) OVER (PARTITION BY orderkey ORDER BY shipdate ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) b " + "FROM lineitem WHERE (partkey = 272 OR partkey = 273) AND suppkey > 50 " + ")" + "SELECT * FROM foo, bar WHERE foo.a = bar.b"; - ExpectedValueProvider leftSpecification = specification( - ImmutableList.of(ORDERKEY_ALIAS), - ImmutableList.of(SHIPDATE_ALIAS, QUANTITY_ALIAS), - ImmutableMap.of(SHIPDATE_ALIAS, SortOrder.ASC_NULLS_LAST, QUANTITY_ALIAS, SortOrder.DESC_NULLS_LAST)); - - ExpectedValueProvider rightSpecification = specification( - ImmutableList.of(rOrderkeyAlias), - ImmutableList.of(rShipdateAlias, rQuantityAlias), - ImmutableMap.of(rShipdateAlias, SortOrder.ASC_NULLS_LAST, rQuantityAlias, SortOrder.DESC_NULLS_LAST)); - - // Too many items in the map to call ImmutableMap.of() :-( - ImmutableMap.Builder leftTableScanBuilder = ImmutableMap.builder(); - leftTableScanBuilder.put(DISCOUNT_ALIAS, "discount"); - leftTableScanBuilder.put(ORDERKEY_ALIAS, "orderkey"); - leftTableScanBuilder.put("PARTKEY", "partkey"); - leftTableScanBuilder.put(SUPPKEY_ALIAS, "suppkey"); - leftTableScanBuilder.put(QUANTITY_ALIAS, "quantity"); - leftTableScanBuilder.put(SHIPDATE_ALIAS, "shipdate"); - - PlanMatchPattern leftTableScan = tableScan("lineitem", leftTableScanBuilder.build()); - - PlanMatchPattern rightTableScan = tableScan( - "lineitem", - ImmutableMap.of( - rOrderkeyAlias, "orderkey", - "R_PARTKEY", "partkey", - "R_SUPPKEY", "suppkey", - rQuantityAlias, "quantity", - rShipdateAlias, "shipdate")); - assertUnitPlan(sql, anyTree( - filter("SUM = AVG", - join(JoinNode.Type.INNER, ImmutableList.of(), - any( - window(leftSpecification, ImmutableMap.of("SUM", functionCall("sum", COMMON_FRAME, ImmutableList.of(DISCOUNT_ALIAS))), - any( - leftTableScan))), - any( - window(rightSpecification, ImmutableMap.of("AVG", functionCall("avg", COMMON_FRAME, ImmutableList.of(rQuantityAlias))), - any( - rightTableScan))))))); + join(JoinNode.Type.INNER, ImmutableList.of(), + any( + window(ImmutableList.of(functionCall("sum", "*")), + anyTree())), + any( + window(ImmutableList.of(functionCall("avg", "*")), + anyTree()))))); } @Test public void testNotMergeDifferentPartition() { @Language("SQL") String sql = "SELECT " + - "SUM(discount) OVER (PARTITION BY suppkey ORDER BY orderkey ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) sum_extendedprice_A, " + + "SUM(extendedprice) OVER (PARTITION BY suppkey ORDER BY orderkey ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) sum_extendedprice_A, " + "SUM(quantity) over (PARTITION BY quantity ORDER BY orderkey ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) sum_quantity_C " + "FROM lineitem"; - ExpectedValueProvider specificationC = specification( - ImmutableList.of(QUANTITY_ALIAS), - ImmutableList.of(ORDERKEY_ALIAS), - ImmutableMap.of(ORDERKEY_ALIAS, SortOrder.ASC_NULLS_LAST)); + Window windowC = new Window( + ImmutableList.of(new SymbolReference("quantity")), + ImmutableList.of(new SortItem(new SymbolReference("orderkey"), SortItem.Ordering.ASCENDING, SortItem.NullOrdering.UNDEFINED)), + Optional.of(commonFrame)); assertUnitPlan(sql, anyTree( - window(specificationC, ImmutableList.of( - functionCall("sum", COMMON_FRAME, ImmutableList.of(QUANTITY_ALIAS))), - window(specificationA, ImmutableList.of( - functionCall("sum", COMMON_FRAME, ImmutableList.of(DISCOUNT_ALIAS))), - LINEITEM_TABLESCAN_DOQS)))); + window(ImmutableList.of( + functionCall("sum", windowC, false, "quantity")), + window(ImmutableList.of( + functionCall("sum", windowA, false, "extendedprice")), + anyNot(WindowNode.class))))); } @Test public void testNotMergeDifferentOrderBy() { @Language("SQL") String sql = "SELECT " + - "SUM(discount) OVER (PARTITION BY suppkey ORDER BY orderkey ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) sum_extendedprice_A, " + + "SUM(extendedprice) OVER (PARTITION BY suppkey ORDER BY orderkey ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) sum_extendedprice_A, " + "SUM(quantity) OVER (PARTITION BY suppkey ORDER BY quantity ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) sum_quantity_C " + "FROM lineitem"; - ExpectedValueProvider specificationC = specification( - ImmutableList.of(SUPPKEY_ALIAS), - ImmutableList.of(QUANTITY_ALIAS), - ImmutableMap.of(QUANTITY_ALIAS, SortOrder.ASC_NULLS_LAST)); + Window windowC = new Window( + ImmutableList.of(new SymbolReference("suppkey")), + ImmutableList.of(new SortItem(new SymbolReference("quantity"), SortItem.Ordering.ASCENDING, SortItem.NullOrdering.UNDEFINED)), + Optional.of(commonFrame)); assertUnitPlan(sql, anyTree( - window(specificationC, ImmutableList.of( - functionCall("sum", COMMON_FRAME, ImmutableList.of(QUANTITY_ALIAS))), - window(specificationA, ImmutableList.of( - functionCall("sum", COMMON_FRAME, ImmutableList.of(DISCOUNT_ALIAS))), - LINEITEM_TABLESCAN_DOQS)))); + window(ImmutableList.of( + functionCall("sum", windowC, false, "quantity")), + window(ImmutableList.of( + functionCall("sum", windowA, false, "extendedprice")), + anyNot(WindowNode.class))))); } @Test @@ -430,19 +369,19 @@ public void testNotMergeDifferentOrdering() "SUM(discount) over (PARTITION BY suppkey ORDER BY orderkey ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) sum_discount_A " + "FROM lineitem"; - ExpectedValueProvider specificationC = specification( - ImmutableList.of(SUPPKEY_ALIAS), - ImmutableList.of(ORDERKEY_ALIAS), - ImmutableMap.of(ORDERKEY_ALIAS, SortOrder.DESC_NULLS_LAST)); + Window windowC = new Window( + ImmutableList.of(new SymbolReference("suppkey")), + ImmutableList.of(new SortItem(new SymbolReference("orderkey"), SortItem.Ordering.DESCENDING, SortItem.NullOrdering.UNDEFINED)), + Optional.of(commonFrame)); assertUnitPlan(sql, anyTree( - window(specificationC, ImmutableList.of( - functionCall("sum", COMMON_FRAME, ImmutableList.of(QUANTITY_ALIAS))), - window(specificationA, ImmutableList.of( - functionCall("sum", COMMON_FRAME, ImmutableList.of(EXTENDEDPRICE_ALIAS)), - functionCall("sum", COMMON_FRAME, ImmutableList.of(DISCOUNT_ALIAS))), - LINEITEM_TABLESCAN_DEOQS)))); + window(ImmutableList.of( + functionCall("sum", windowC, false, "quantity")), + window(ImmutableList.of( + functionCall("sum", windowA, false, "extendedprice"), + functionCall("sum", windowA, false, "discount")), + anyNot(WindowNode.class))))); } @Test @@ -454,33 +393,40 @@ public void testNotMergeDifferentNullOrdering() "SUM(discount) OVER (PARTITION BY suppkey ORDER BY orderkey ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) sum_discount_A " + "FROM lineitem"; - ExpectedValueProvider specificationC = specification( - ImmutableList.of(SUPPKEY_ALIAS), - ImmutableList.of(ORDERKEY_ALIAS), - ImmutableMap.of(ORDERKEY_ALIAS, SortOrder.ASC_NULLS_FIRST)); + Window windowC = new Window( + ImmutableList.of(new SymbolReference("suppkey")), + ImmutableList.of(new SortItem(new SymbolReference("orderkey"), SortItem.Ordering.ASCENDING, SortItem.NullOrdering.FIRST)), + Optional.of(commonFrame)); assertUnitPlan(sql, anyTree( - window(specificationC, ImmutableList.of( - functionCall("sum", COMMON_FRAME, ImmutableList.of(QUANTITY_ALIAS))), - window(specificationA, ImmutableList.of( - functionCall("sum", COMMON_FRAME, ImmutableList.of(EXTENDEDPRICE_ALIAS)), - functionCall("sum", COMMON_FRAME, ImmutableList.of(DISCOUNT_ALIAS))), - LINEITEM_TABLESCAN_DEOQS)))); + window(ImmutableList.of( + functionCall("sum", windowC, false, "quantity")), + window(ImmutableList.of( + functionCall("sum", windowA, false, "extendedprice"), + functionCall("sum", windowA, false, "discount")), + anyNot(WindowNode.class))))); } private void assertUnitPlan(@Language("SQL") String sql, PlanMatchPattern pattern) { - LocalQueryRunner queryRunner = getQueryRunner(); - List optimizers = ImmutableList.of( - new UnaliasSymbolReferences(), - new PruneIdentityProjections(), - new MergeWindows(), - new PruneUnreferencedOutputs()); + Plan actualPlan = unitPlan(sql); queryRunner.inTransaction(transactionSession -> { - Plan actualPlan = queryRunner.createPlan(transactionSession, sql, optimizers); PlanAssert.assertPlan(transactionSession, queryRunner.getMetadata(), actualPlan, pattern); return null; }); } + + private Plan unitPlan(@Language("SQL") String sql) + { + FeaturesConfig featuresConfig = new FeaturesConfig() + .setDistributedIndexJoinsEnabled(false) + .setOptimizeHashGeneration(true); + List optimizers = ImmutableList.of( + new UnaliasSymbolReferences(), + new PruneIdentityProjections(), + new MergeWindows(), + new PruneUnreferencedOutputs()); + return queryRunner.inTransaction(transactionSession -> queryRunner.createPlan(transactionSession, sql, featuresConfig, optimizers)); + } } diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestMixedDistinctAggregationOptimizer.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestMixedDistinctAggregationOptimizer.java index 069eb74bc403..071d441bed1e 100644 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestMixedDistinctAggregationOptimizer.java +++ b/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestMixedDistinctAggregationOptimizer.java @@ -15,9 +15,9 @@ import com.facebook.presto.Session; import com.facebook.presto.SystemSessionProperties; +import com.facebook.presto.sql.analyzer.FeaturesConfig; import com.facebook.presto.sql.planner.Plan; import com.facebook.presto.sql.planner.Symbol; -import com.facebook.presto.sql.planner.assertions.ExpectedValueProvider; import com.facebook.presto.sql.planner.assertions.PlanAssert; import com.facebook.presto.sql.planner.assertions.PlanMatchPattern; import com.facebook.presto.sql.tree.FunctionCall; @@ -29,16 +29,13 @@ import org.testng.annotations.Test; import java.util.List; -import java.util.Map; import java.util.Optional; import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.aggregation; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.anySymbol; import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.anyTree; import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.functionCall; import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.groupingSet; import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.project; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.tableScan; import static com.facebook.presto.testing.TestingSession.testSessionBuilder; import static com.facebook.presto.tpch.TpchMetadata.TINY_SCHEMA_NAME; @@ -57,7 +54,7 @@ public TestMixedDistinctAggregationOptimizer() this.queryRunner = new LocalQueryRunner(defaultSession); queryRunner.createCatalog(queryRunner.getDefaultSession().getCatalog().get(), new TpchConnectorFactory(1), - ImmutableMap.of()); + ImmutableMap.of()); } @Test @@ -73,16 +70,13 @@ public void testMixedDistinctAggregationOptimizer() // Second Aggregation data List groupByKeysSecond = ImmutableList.of(groupBy); - Map, ExpectedValueProvider> aggregationsSecond = ImmutableMap.of( - Optional.of("arbitrary"), PlanMatchPattern.functionCall("arbitrary", false, ImmutableList.of(anySymbol())), - Optional.of("count"), PlanMatchPattern.functionCall("count", false, ImmutableList.of(anySymbol()))); + List aggregationsSecond = ImmutableList.of( + functionCall("arbitrary", "*"), + functionCall("count", "*")); // First Aggregation data List groupByKeysFirst = ImmutableList.of(groupBy, distinctAggregation, group); - Map, ExpectedValueProvider> aggregationsFirst = ImmutableMap.of( - Optional.of("MAX"), functionCall("max", ImmutableList.of("TOTALPRICE"))); - - PlanMatchPattern tableScan = tableScan("orders", ImmutableMap.of("TOTALPRICE", "totalprice")); + List aggregationsFirst = ImmutableList.of(functionCall("max", "totalprice")); // GroupingSet symbols ImmutableList.Builder> groups = ImmutableList.builder(); @@ -93,16 +87,16 @@ public void testMixedDistinctAggregationOptimizer() project( aggregation(ImmutableList.of(groupByKeysFirst), aggregationsFirst, ImmutableMap.of(), Optional.empty(), groupingSet(groups.build(), - anyTree(tableScan)))))); + anyTree()))))); List optimizerProvider = ImmutableList.of( new UnaliasSymbolReferences(), new PruneIdentityProjections(), new OptimizeMixedDistinctAggregations(queryRunner.getMetadata()), new PruneUnreferencedOutputs()); + Plan actualPlan = queryRunner.inTransaction(transactionSession -> queryRunner.createPlan(transactionSession, sql, new FeaturesConfig(), optimizerProvider)); queryRunner.inTransaction(transactionSession -> { - Plan actualPlan = queryRunner.createPlan(transactionSession, sql, optimizerProvider); PlanAssert.assertPlan(transactionSession, queryRunner.getMetadata(), actualPlan, expectedPlanPattern); return null; }); diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestSetFlatteningOptimizer.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestSetFlatteningOptimizer.java deleted file mode 100644 index 32f65312d1e3..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/optimizations/TestSetFlatteningOptimizer.java +++ /dev/null @@ -1,132 +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.sql.planner.optimizations; - -import com.facebook.presto.sql.planner.assertions.BasePlanTest; -import com.facebook.presto.sql.planner.assertions.PlanMatchPattern; -import com.google.common.collect.ImmutableList; -import org.testng.annotations.Test; - -import java.util.List; - -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.anyTree; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.except; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.intersect; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.tableScan; -import static com.facebook.presto.sql.planner.assertions.PlanMatchPattern.union; - -public class TestSetFlatteningOptimizer - extends BasePlanTest -{ - @Test - public void testFlattensUnion() - { - assertPlan( - "(SELECT * FROM nation UNION SELECT * FROM nation)" + - "UNION (SELECT * FROM nation UNION SELECT * FROM nation)", - anyTree( - union( - tableScan("nation"), - tableScan("nation"), - tableScan("nation"), - tableScan("nation")))); - } - - @Test - public void testFlattensUnionAll() - { - assertPlan( - "(SELECT * FROM nation UNION ALL SELECT * FROM nation)" + - "UNION ALL (SELECT * FROM nation UNION ALL SELECT * FROM nation)", - anyTree( - union( - tableScan("nation"), - tableScan("nation"), - tableScan("nation"), - tableScan("nation")))); - } - - @Test - public void testFlattensUnionAndUnionAllWhenAllowed() - { - assertPlan( - "SELECT * FROM nation " + - "UNION ALL (SELECT * FROM nation " + - "UNION (SELECT * FROM nation UNION ALL select * FROM nation))", - anyTree( - union( - tableScan("nation"), - anyTree( - union( - tableScan("nation"), - tableScan("nation"), - tableScan("nation")))))); - } - - @Test - public void testFlattensIntersect() - { - assertPlan( - "(SELECT * FROM nation INTERSECT SELECT * FROM nation)" + - "INTERSECT (SELECT * FROM nation INTERSECT SELECT * FROM nation)", - anyTree( - intersect( - tableScan("nation"), - tableScan("nation"), - tableScan("nation"), - tableScan("nation")))); - } - - @Test - public void testFlattensOnlyFirstInputOfExcept() - { - assertPlan( - "(SELECT * FROM nation EXCEPT SELECT * FROM nation)" + - "EXCEPT (SELECT * FROM nation EXCEPT SELECT * FROM nation)", - anyTree( - except( - tableScan("nation"), - tableScan("nation"), - except( - tableScan("nation"), - tableScan("nation"))))); - } - - @Test - public void testDoesNotFlattenDifferentSetOperations() - { - assertPlan( - "(SELECT * FROM nation EXCEPT SELECT * FROM nation)" + - "UNION (SELECT * FROM nation INTERSECT SELECT * FROM nation)", - anyTree( - union( - except( - tableScan("nation"), - tableScan("nation")), - intersect( - tableScan("nation"), - tableScan("nation"))))); - } - - public void assertPlan(String sql, PlanMatchPattern pattern) - { - List optimizers = ImmutableList.of( - new UnaliasSymbolReferences(), - new PruneUnreferencedOutputs(), - new PruneIdentityProjections(), - new SetFlatteningOptimizer()); - assertPlanWithOptimizers(sql, pattern, optimizers); - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/sql/planner/sanity/TestVerifyOnlyOneOutputNode.java b/presto-main/src/test/java/com/facebook/presto/sql/planner/sanity/TestVerifyOnlyOneOutputNode.java deleted file mode 100644 index a9c2b0c25ba5..000000000000 --- a/presto-main/src/test/java/com/facebook/presto/sql/planner/sanity/TestVerifyOnlyOneOutputNode.java +++ /dev/null @@ -1,69 +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.sql.planner.sanity; - -import com.facebook.presto.sql.planner.PlanNodeIdAllocator; -import com.facebook.presto.sql.planner.Symbol; -import com.facebook.presto.sql.planner.plan.ExplainAnalyzeNode; -import com.facebook.presto.sql.planner.plan.OutputNode; -import com.facebook.presto.sql.planner.plan.PlanNode; -import com.facebook.presto.sql.planner.plan.ProjectNode; -import com.facebook.presto.sql.planner.plan.ValuesNode; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import org.testng.annotations.Test; - -public class TestVerifyOnlyOneOutputNode -{ - private final PlanNodeIdAllocator idAllocator = new PlanNodeIdAllocator(); - - @Test - public void testValidateSuccessful() - throws Exception - { - // random seemingly valid plan - PlanNode root = - new OutputNode(idAllocator.getNextId(), - new ProjectNode(idAllocator.getNextId(), - new ValuesNode( - idAllocator.getNextId(), ImmutableList.of(), ImmutableList.of() - ), - ImmutableMap.of() - ), ImmutableList.of(), ImmutableList.of() - ); - new VerifyOnlyOneOutputNode().validate(root, null, null, null, null); - } - - @Test(expectedExceptions = IllegalStateException.class) - public void testValidateFailed() - throws Exception - { - // random plan with 2 output nodes - PlanNode root = - new OutputNode(idAllocator.getNextId(), - new ExplainAnalyzeNode(idAllocator.getNextId(), - new OutputNode(idAllocator.getNextId(), - new ProjectNode(idAllocator.getNextId(), - new ValuesNode( - idAllocator.getNextId(), ImmutableList.of(), ImmutableList.of() - ), - ImmutableMap.of() - ), ImmutableList.of(), ImmutableList.of() - ), new Symbol("a") - ), - ImmutableList.of(), ImmutableList.of() - ); - new VerifyOnlyOneOutputNode().validate(root, null, null, null, null); - } -} diff --git a/presto-main/src/test/java/com/facebook/presto/type/TestArrayOperators.java b/presto-main/src/test/java/com/facebook/presto/type/TestArrayOperators.java index 4159566146b4..eb82bbc6cab6 100644 --- a/presto-main/src/test/java/com/facebook/presto/type/TestArrayOperators.java +++ b/presto-main/src/test/java/com/facebook/presto/type/TestArrayOperators.java @@ -440,13 +440,14 @@ public void testSubscript() public void testElementAt() throws Exception { + String outOfBounds = "Array subscript out of bounds"; + assertInvalidFunction("ELEMENT_AT(ARRAY [], -1)", outOfBounds); assertInvalidFunction("ELEMENT_AT(ARRAY [], 0)", "SQL array indices start at 1"); + assertInvalidFunction("ELEMENT_AT(ARRAY [], 1)", outOfBounds); assertInvalidFunction("ELEMENT_AT(ARRAY [1, 2, 3], 0)", "SQL array indices start at 1"); + assertInvalidFunction("ELEMENT_AT(ARRAY [1, 2, 3], 4)", outOfBounds); + assertInvalidFunction("ELEMENT_AT(ARRAY [1, 2, 3], -4)", outOfBounds); - assertFunction("ELEMENT_AT(ARRAY [], 1)", UNKNOWN, null); - assertFunction("ELEMENT_AT(ARRAY [], -1)", UNKNOWN, null); - assertFunction("ELEMENT_AT(ARRAY [1, 2, 3], 4)", INTEGER, null); - assertFunction("ELEMENT_AT(ARRAY [1, 2, 3], -4)", INTEGER, null); assertFunction("ELEMENT_AT(ARRAY [NULL], 1)", UNKNOWN, null); assertFunction("ELEMENT_AT(ARRAY [NULL], -1)", UNKNOWN, null); assertFunction("ELEMENT_AT(ARRAY [NULL, NULL, NULL], 3)", UNKNOWN, null); @@ -885,7 +886,6 @@ public void testSequenceDateTimeYearToMonth() assertInvalidFunction("SEQUENCE(date '2016-04-12', date '2016-06-12', interval '-1' month)", INVALID_FUNCTION_ARGUMENT); } - @Override public void assertInvalidFunction(String projection, SemanticErrorCode errorCode) { try { @@ -978,12 +978,12 @@ private void assertArrayHashOperator(String inputArray, Type elementType, List cast('foo' as char(3))", BOOLEAN, false); assertFunction("cast('foo' as char(3)) <> cast('bar' as char(3))", BOOLEAN, true); assertFunction("cast('bar' as char(3)) <> cast('foo' as char(3))", BOOLEAN, true); - - assertFunction("cast('a' as char(2)) <> cast('a ' as char(2))", BOOLEAN, false); - assertFunction("cast('a ' as char(2)) <> cast('a' as char(2))", BOOLEAN, false); - - assertFunction("cast('a' as char(3)) <> cast('a' as char(2))", BOOLEAN, true); - assertFunction("cast('' as char(3)) <> cast('' as char(2))", BOOLEAN, true); - assertFunction("cast('' as char(2)) <> cast('' as char(2))", BOOLEAN, false); } @Test @@ -70,12 +56,6 @@ public void testLessThan() assertFunction("cast('bar' as char(3)) < cast('foo' as char(3))", BOOLEAN, true); assertFunction("cast('foobar' as char(6)) < cast('foobaz' as char(6))", BOOLEAN, true); assertFunction("cast('foob r' as char(6)) < cast('foobar' as char(6))", BOOLEAN, true); - assertFunction("cast('\0' as char(1)) < cast(' ' as char(1))", BOOLEAN, true); - assertFunction("cast('\0' as char(1)) < cast('' as char(0))", BOOLEAN, false); // length mismatch, coercion to VARCHAR applies, thus '\0' > '' - assertFunction("cast('abc\0' as char(4)) < cast('abc' as char(4))", BOOLEAN, true); // 'abc' is implicitly padded with spaces -> 'abc' is greater - assertFunction("cast('\0' as char(1)) < cast('\0 ' as char(2))", BOOLEAN, true); // length mismatch, coercion to VARCHAR applies - assertFunction("cast('\0' as char(2)) < cast('\0 ' as char(2))", BOOLEAN, false); // '\0' is implicitly padded with spaces -> both are equal - assertFunction("cast('\0 a' as char(3)) < cast('\0' as char(3))", BOOLEAN, false); } @Test @@ -93,12 +73,6 @@ public void testLessThanOrEqual() assertFunction("cast('bar' as char(3)) <= cast('foo' as char(3))", BOOLEAN, true); assertFunction("cast('foobar' as char(6)) <= cast('foobaz' as char(6))", BOOLEAN, true); assertFunction("cast('foob r' as char(6)) <= cast('foobar' as char(6))", BOOLEAN, true); - assertFunction("cast('\0' as char(1)) <= cast(' ' as char(1))", BOOLEAN, true); - assertFunction("cast('\0' as char(1)) <= cast('' as char(0))", BOOLEAN, false); // length mismatch, coercion to VARCHAR applies, thus '\0' > '' - assertFunction("cast('abc\0' as char(4)) <= cast('abc' as char(4))", BOOLEAN, true); // 'abc' is implicitly padded with spaces -> 'abc' is greater - assertFunction("cast('\0' as char(1)) <= cast('\0 ' as char(2))", BOOLEAN, true); // length mismatch, coercion to VARCHAR applies - assertFunction("cast('\0' as char(2)) <= cast('\0 ' as char(2))", BOOLEAN, true); // '\0' is implicitly padded with spaces -> both are equal - assertFunction("cast('\0 a' as char(3)) <= cast('\0' as char(3))", BOOLEAN, false); } @Test @@ -116,12 +90,6 @@ public void testGreaterThan() assertFunction("cast('bar' as char(3)) > cast('foo' as char(3))", BOOLEAN, false); assertFunction("cast('foobar' as char(6)) > cast('foobaz' as char(6))", BOOLEAN, false); assertFunction("cast('foob r' as char(6)) > cast('foobar' as char(6))", BOOLEAN, false); - assertFunction("cast(' ' as char(1)) > cast('\0' as char(1))", BOOLEAN, true); - assertFunction("cast('' as char(0)) > cast('\0' as char(1))", BOOLEAN, false); // length mismatch, coercion to VARCHAR applies, thus '\0' > '' - assertFunction("cast('abc' as char(4)) > cast('abc\0' as char(4))", BOOLEAN, true); // 'abc' is implicitly padded with spaces -> 'abc' is greater - assertFunction("cast('\0 ' as char(2)) > cast('\0' as char(1))", BOOLEAN, true); // length mismatch, coercion to VARCHAR applies - assertFunction("cast('\0 ' as char(2)) > cast('\0' as char(2))", BOOLEAN, false); // '\0' is implicitly padded with spaces -> both are equal - assertFunction("cast('\0 a' as char(3)) > cast('\0' as char(3))", BOOLEAN, true); } @Test @@ -139,12 +107,6 @@ public void testGreaterThanOrEqual() assertFunction("cast('bar' as char(3)) >= cast('foo' as char(3))", BOOLEAN, false); assertFunction("cast('foobar' as char(6)) >= cast('foobaz' as char(6))", BOOLEAN, false); assertFunction("cast('foob r' as char(6)) >= cast('foobar' as char(6))", BOOLEAN, false); - assertFunction("cast(' ' as char(1)) >= cast('\0' as char(1))", BOOLEAN, true); - assertFunction("cast('' as char(0)) >= cast('\0' as char(1))", BOOLEAN, false); // length mismatch, coercion to VARCHAR applies, thus '\0' > '' - assertFunction("cast('abc' as char(4)) >= cast('abc\0' as char(4))", BOOLEAN, true); // 'abc' is implicitly padded with spaces -> 'abc' is greater - assertFunction("cast('\0 ' as char(2)) >= cast('\0' as char(1))", BOOLEAN, true); // length mismatch, coercion to VARCHAR applies - assertFunction("cast('\0 ' as char(2)) >= cast('\0' as char(2))", BOOLEAN, true); // '\0' is implicitly padded with spaces -> both are equal - assertFunction("cast('\0 a' as char(3)) >= cast('\0' as char(3))", BOOLEAN, true); } @Test @@ -166,8 +128,6 @@ public void testBetween() assertFunction("cast('bar' as char(3)) BETWEEN cast('bar' as char(3)) AND cast('foo' as char(3))", BOOLEAN, true); assertFunction("cast('bar' as char(3)) BETWEEN cast('bar' as char(3)) AND cast('bar' as char(3))", BOOLEAN, true); - assertFunction("cast('\0 a' as char(3)) BETWEEN cast('\0' as char(3)) AND cast('\0a' as char(3))", BOOLEAN, true); - // length based comparison assertFunction("cast('bar' as char(4)) BETWEEN cast('bar' as char(3)) AND cast('bar' as char(5))", BOOLEAN, true); assertFunction("cast('bar' as char(4)) BETWEEN cast('bar' as char(5)) AND cast('bar' as char(7))", BOOLEAN, false); diff --git a/presto-main/src/test/java/com/facebook/presto/type/TestDecimalCasts.java b/presto-main/src/test/java/com/facebook/presto/type/TestDecimalCasts.java index 5e6c631f30ea..f28a83cc2d83 100644 --- a/presto-main/src/test/java/com/facebook/presto/type/TestDecimalCasts.java +++ b/presto-main/src/test/java/com/facebook/presto/type/TestDecimalCasts.java @@ -254,13 +254,6 @@ public void testDoubleToDecimalCasts() assertInvalidCast("CAST(DOUBLE '1000.01' AS DECIMAL(5,2))", "Cannot cast DOUBLE '1000.01' to DECIMAL(5, 2)"); assertInvalidCast("CAST(DOUBLE '-234.0' AS DECIMAL(2,0))", "Cannot cast DOUBLE '-234.0' to DECIMAL(2, 0)"); assertInvalidCast("CAST(DOUBLE '12345678901.1' AS DECIMAL(20, 10))", "Cannot cast DOUBLE '1.23456789011E10' to DECIMAL(20, 10)"); - - assertInvalidCast("CAST(nan() AS DECIMAL(10,5))", "Cannot cast floating point NaN to DECIMAL(10, 5)"); - assertInvalidCast("CAST(infinity() AS DECIMAL(10,1))", "Cannot cast floating point Infinity to DECIMAL(10, 1)"); - assertInvalidCast("CAST(-infinity() AS DECIMAL(1,1))", "Cannot cast floating point -Infinity to DECIMAL(1, 1)"); - assertInvalidCast("CAST(nan() AS DECIMAL(38,10))", "Cannot cast floating point NaN to DECIMAL(38, 10)"); - assertInvalidCast("CAST(infinity() AS DECIMAL(38,2))", "Cannot cast floating point Infinity to DECIMAL(38, 2)"); - assertInvalidCast("CAST(-infinity() AS DECIMAL(38,1))", "Cannot cast floating point -Infinity to DECIMAL(38, 1)"); } @Test @@ -300,13 +293,6 @@ public void testFloatToDecimalCasts() assertInvalidCast("CAST(REAL '1000.01' AS DECIMAL(5,2))", "Cannot cast REAL '1000.01' to DECIMAL(5, 2)"); assertInvalidCast("CAST(REAL '-234.0' AS DECIMAL(2,0))", "Cannot cast REAL '-234.0' to DECIMAL(2, 0)"); assertInvalidCast("CAST(REAL '98765430784.0' AS DECIMAL(20, 10))", "Cannot cast REAL '9.8765431E10' to DECIMAL(20, 10)"); - - assertInvalidCast("CAST(CAST(nan() as REAL) AS DECIMAL(10,5))", "Cannot cast floating point NaN to DECIMAL(10, 5)"); - assertInvalidCast("CAST(CAST(infinity() as REAL) AS DECIMAL(10,1))", "Cannot cast floating point Infinity to DECIMAL(10, 1)"); - assertInvalidCast("CAST(CAST(-infinity() as REAL) AS DECIMAL(1,1))", "Cannot cast floating point -Infinity to DECIMAL(1, 1)"); - assertInvalidCast("CAST(CAST(nan() as REAL) AS DECIMAL(38,10))", "Cannot cast floating point NaN to DECIMAL(38, 10)"); - assertInvalidCast("CAST(CAST(infinity() as REAL) AS DECIMAL(38,2))", "Cannot cast floating point Infinity to DECIMAL(38, 2)"); - assertInvalidCast("CAST(CAST(-infinity() as REAL) AS DECIMAL(38,1))", "Cannot cast floating point -Infinity to DECIMAL(38, 1)"); } @Test diff --git a/presto-main/src/test/java/com/facebook/presto/type/TestMapOperators.java b/presto-main/src/test/java/com/facebook/presto/type/TestMapOperators.java index 2538f9c11843..c93d93f4ad55 100644 --- a/presto-main/src/test/java/com/facebook/presto/type/TestMapOperators.java +++ b/presto-main/src/test/java/com/facebook/presto/type/TestMapOperators.java @@ -81,7 +81,7 @@ public void testStackRepresentation() DynamicSliceOutput actualSliceOutput = new DynamicSliceOutput(100); writeBlock(actualSliceOutput, actualBlock); - Block expectedBlock = new InterleavedBlockBuilder(ImmutableList.of(DOUBLE, new ArrayType(BIGINT)), new BlockBuilderStatus(), 3) + Block expectedBlock = new InterleavedBlockBuilder(ImmutableList.of(DOUBLE, new ArrayType(BIGINT)), new BlockBuilderStatus(), 3) .writeLong(doubleToLongBits(1.0)) .closeEntry() .writeObject( @@ -133,12 +133,6 @@ public void testConstructor() assertInvalidFunction("MAP(ARRAY [1], ARRAY [2, 4])", "Key and value arrays must be the same length"); } - @Test - public void testEmptyMapConstructor() - { - assertFunction("MAP()", new MapType(UNKNOWN, UNKNOWN), ImmutableMap.of()); - } - @Test public void testCardinality() throws Exception diff --git a/presto-main/src/test/java/com/facebook/presto/type/TestRowOperators.java b/presto-main/src/test/java/com/facebook/presto/type/TestRowOperators.java index cd3caaf5be57..5161d6ca4022 100644 --- a/presto-main/src/test/java/com/facebook/presto/type/TestRowOperators.java +++ b/presto-main/src/test/java/com/facebook/presto/type/TestRowOperators.java @@ -183,7 +183,7 @@ public void testRowEquality() fail("hyperloglog is not comparable"); } catch (SemanticException e) { - if (!e.getMessage().matches("\\Qline 1:81: '=' cannot be applied to row(col0 HyperLogLog), row(col0 HyperLogLog)\\E")) { + if (!e.getMessage().matches("\\Qline 1:81: '=' cannot be applied to row(COL0 HyperLogLog), row(COL0 HyperLogLog)\\E")) { throw e; } //Expected diff --git a/presto-main/src/test/java/com/facebook/presto/type/TestTypeRegistry.java b/presto-main/src/test/java/com/facebook/presto/type/TestTypeRegistry.java index 4906469a9cf9..838125fa9eef 100644 --- a/presto-main/src/test/java/com/facebook/presto/type/TestTypeRegistry.java +++ b/presto-main/src/test/java/com/facebook/presto/type/TestTypeRegistry.java @@ -294,7 +294,7 @@ private Set getStandardPrimitiveTypes() return builder.build(); } - private static void assertCommonSuperType(Type firstType, Type secondType, Type expected) + private void assertCommonSuperType(Type firstType, Type secondType, Type expected) { TypeRegistry typeManager = new TypeRegistry(); assertEquals(typeManager.getCommonSuperType(firstType, secondType), Optional.ofNullable(expected)); diff --git a/presto-main/src/test/java/com/facebook/presto/util/TestPowerOfTwoValidator.java b/presto-main/src/test/java/com/facebook/presto/util/TestPowerOfTwoValidator.java index 227133ced1ca..d985dd4b3ddf 100644 --- a/presto-main/src/test/java/com/facebook/presto/util/TestPowerOfTwoValidator.java +++ b/presto-main/src/test/java/com/facebook/presto/util/TestPowerOfTwoValidator.java @@ -130,7 +130,7 @@ public Integer getConstrainedByPowerOfTwoBoxed() public static class NullPowerOfTwoAnnotation { @PowerOfTwo - public static Integer getConstrainedByPowerOfTwo() + public Integer getConstrainedByPowerOfTwo() { return null; } diff --git a/presto-ml/src/test/java/com/facebook/presto/ml/TestMLQueries.java b/presto-ml/src/test/java/com/facebook/presto/ml/TestMLQueries.java index 6f92f99418f8..cf64a19201b7 100644 --- a/presto-ml/src/test/java/com/facebook/presto/ml/TestMLQueries.java +++ b/presto-ml/src/test/java/com/facebook/presto/ml/TestMLQueries.java @@ -64,7 +64,7 @@ private static LocalQueryRunner createLocalQueryRunner() localQueryRunner.createCatalog( defaultSession.getCatalog().get(), new TpchConnectorFactory(1), - ImmutableMap.of()); + ImmutableMap.of()); MLPlugin plugin = new MLPlugin(); for (Type type : plugin.getTypes()) { diff --git a/presto-mongodb/src/main/java/com/facebook/presto/mongodb/MongoMetadata.java b/presto-mongodb/src/main/java/com/facebook/presto/mongodb/MongoMetadata.java index 207ed67e2819..994b05049eb3 100644 --- a/presto-mongodb/src/main/java/com/facebook/presto/mongodb/MongoMetadata.java +++ b/presto-mongodb/src/main/java/com/facebook/presto/mongodb/MongoMetadata.java @@ -182,7 +182,7 @@ public ConnectorTableLayout getTableLayout(ConnectorSession session, ConnectorTa MongoTableLayoutHandle layout = checkType(handle, MongoTableLayoutHandle.class, "layout"); // tables in this connector have a single layout - return getTableLayouts(session, layout.getTable(), Constraint.alwaysTrue(), Optional.empty()) + return getTableLayouts(session, layout.getTable(), Constraint.alwaysTrue(), Optional.empty()) .get(0) .getTableLayout(); } diff --git a/presto-mongodb/src/main/java/com/facebook/presto/mongodb/MongoPageSink.java b/presto-mongodb/src/main/java/com/facebook/presto/mongodb/MongoPageSink.java index 1c901f5501cc..1f556472afd4 100644 --- a/presto-mongodb/src/main/java/com/facebook/presto/mongodb/MongoPageSink.java +++ b/presto-mongodb/src/main/java/com/facebook/presto/mongodb/MongoPageSink.java @@ -16,26 +16,16 @@ import com.facebook.presto.spi.ConnectorPageSink; import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.Page; -import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.SchemaTableName; -import com.facebook.presto.spi.StandardErrorCode; import com.facebook.presto.spi.block.Block; -import com.facebook.presto.spi.type.BigintType; -import com.facebook.presto.spi.type.BooleanType; -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.DoubleType; -import com.facebook.presto.spi.type.IntegerType; import com.facebook.presto.spi.type.NamedTypeSignature; -import com.facebook.presto.spi.type.SmallintType; -import com.facebook.presto.spi.type.TimeType; -import com.facebook.presto.spi.type.TimestampType; -import com.facebook.presto.spi.type.TimestampWithTimeZoneType; -import com.facebook.presto.spi.type.TinyintType; +import com.facebook.presto.spi.type.SqlDate; +import com.facebook.presto.spi.type.SqlTime; +import com.facebook.presto.spi.type.SqlTimestamp; +import com.facebook.presto.spi.type.SqlTimestampWithTimeZone; +import com.facebook.presto.spi.type.SqlVarbinary; import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.TypeSignatureParameter; -import com.facebook.presto.spi.type.VarbinaryType; import com.google.common.collect.ImmutableList; import com.mongodb.client.MongoCollection; import com.mongodb.client.model.InsertManyOptions; @@ -44,8 +34,6 @@ import org.bson.types.Binary; import org.bson.types.ObjectId; -import java.math.BigDecimal; -import java.math.BigInteger; import java.util.ArrayList; import java.util.Collection; import java.util.Date; @@ -56,15 +44,13 @@ import java.util.concurrent.TimeUnit; import static com.facebook.presto.mongodb.ObjectIdType.OBJECT_ID; +import static com.facebook.presto.mongodb.TypeUtils.containsType; import static com.facebook.presto.mongodb.TypeUtils.isArrayType; import static com.facebook.presto.mongodb.TypeUtils.isMapType; import static com.facebook.presto.mongodb.TypeUtils.isRowType; -import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; -import static com.facebook.presto.spi.type.DateTimeEncoding.unpackMillisUtc; -import static com.facebook.presto.spi.type.Varchars.isVarcharType; -import static java.util.Collections.unmodifiableList; -import static java.util.Collections.unmodifiableMap; +import static com.facebook.presto.spi.type.VarbinaryType.VARBINARY; import static java.util.concurrent.CompletableFuture.completedFuture; +import static java.util.stream.Collectors.toList; public class MongoPageSink implements ConnectorPageSink @@ -73,6 +59,7 @@ public class MongoPageSink private final ConnectorSession session; private final SchemaTableName schemaTableName; private final List columns; + private final List requireTranslate; private final String implicitPrefix; public MongoPageSink(MongoClientConfig config, @@ -85,6 +72,11 @@ public MongoPageSink(MongoClientConfig config, this.session = session; this.schemaTableName = schemaTableName; this.columns = columns; + this.requireTranslate = columns.stream() + .map(c -> containsType(c.getType(), TypeUtils::isDateType, + TypeUtils::isMapType, TypeUtils::isRowType, + OBJECT_ID::equals, VARBINARY::equals)) + .collect(toList()); this.implicitPrefix = config.getImplicitRowFieldPrefix(); } @@ -99,7 +91,7 @@ public CompletableFuture appendPage(Page page) for (int channel = 0; channel < page.getChannelCount(); channel++) { MongoColumnHandle column = columns.get(channel); - doc.append(column.getName(), getObjectValue(columns.get(channel).getType(), page.getBlock(channel), position)); + doc.append(column.getName(), getObjectValue(columns.get(channel).getType(), page.getBlock(channel), position, requireTranslate.get(channel))); } batch.add(doc); } @@ -108,127 +100,89 @@ public CompletableFuture appendPage(Page page) return NOT_BLOCKED; } - private Object getObjectValue(Type type, Block block, int position) + private Object getObjectValue(Type type, Block block, int position, boolean translate) { if (block.isNull(position)) { - if (type.equals(OBJECT_ID)) { - return new ObjectId(); - } return null; } - if (type.equals(OBJECT_ID)) { - return new ObjectId(block.getSlice(position, 0, block.getLength(position)).getBytes()); - } - if (type.equals(BooleanType.BOOLEAN)) { - return type.getBoolean(block, position); - } - if (type.equals(BigintType.BIGINT)) { - return type.getLong(block, position); - } - if (type.equals(IntegerType.INTEGER)) { - return (int) type.getLong(block, position); - } - if (type.equals(SmallintType.SMALLINT)) { - return (short) type.getLong(block, position); - } - if (type.equals(TinyintType.TINYINT)) { - return (byte) type.getLong(block, position); - } - if (type.equals(DoubleType.DOUBLE)) { - return type.getDouble(block, position); - } - if (isVarcharType(type)) { - return type.getSlice(block, position).toStringUtf8(); - } - if (type.equals(VarbinaryType.VARBINARY)) { - return new Binary(type.getSlice(block, position).getBytes()); - } - if (type.equals(DateType.DATE)) { - long days = type.getLong(block, position); - return new Date(TimeUnit.DAYS.toMillis(days)); - } - if (type.equals(TimeType.TIME)) { - long millisUtc = type.getLong(block, position); - return new Date(millisUtc); + Object value = type.getObjectValue(session, block, position); + + if (translate) { + value = translateValue(type, value); } - if (type.equals(TimestampType.TIMESTAMP)) { - long millisUtc = type.getLong(block, position); - return new Date(millisUtc); + + return value; + } + + private Object translateValue(Type type, Object value) + { + if (type.equals(OBJECT_ID)) { + value = value == null ? new ObjectId() : new ObjectId(((SqlVarbinary) value).getBytes()); } - if (type.equals(TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE)) { - long millisUtc = unpackMillisUtc(type.getLong(block, position)); - return new Date(millisUtc); + + if (value == null) { + return null; } - if (type instanceof DecimalType) { - // TODO: decimal type might not support yet - DecimalType decimalType = (DecimalType) type; - BigInteger unscaledValue; - if (decimalType.isShort()) { - unscaledValue = BigInteger.valueOf(decimalType.getLong(block, position)); + + if (type.getJavaType() == long.class) { + if (value instanceof SqlDate) { + return new Date(TimeUnit.DAYS.toMillis(((SqlDate) value).getDays())); } - else { - unscaledValue = Decimals.decodeUnscaledValue(decimalType.getSlice(block, position)); + if (value instanceof SqlTime) { + return new Date(((SqlTime) value).getMillisUtc()); } - return new BigDecimal(unscaledValue); - } - if (isArrayType(type)) { - Type elementType = type.getTypeParameters().get(0); - - Block arrayBlock = block.getObject(position, Block.class); - - List list = new ArrayList<>(arrayBlock.getPositionCount()); - for (int i = 0; i < arrayBlock.getPositionCount(); i++) { - Object element = getObjectValue(elementType, arrayBlock, i); - list.add(element); + if (value instanceof SqlTimestamp) { + return new Date(((SqlTimestamp) value).getMillisUtc()); + } + if (value instanceof SqlTimestampWithTimeZone) { + return new Date(((SqlTimestampWithTimeZone) value).getMillisUtc()); } - - return unmodifiableList(list); } - if (isMapType(type)) { - Type keyType = type.getTypeParameters().get(0); - Type valueType = type.getTypeParameters().get(1); - - Block mapBlock = block.getObject(position, Block.class); - - // map type is converted into list of fixed keys document - List values = new ArrayList<>(mapBlock.getPositionCount() / 2); - for (int i = 0; i < mapBlock.getPositionCount(); i += 2) { - Map mapValue = new HashMap<>(); - mapValue.put("key", getObjectValue(keyType, mapBlock, i)); - mapValue.put("value", getObjectValue(valueType, mapBlock, i + 1)); - values.add(mapValue); + else if (type.getJavaType() == Slice.class) { + if (type.equals(VARBINARY)) { + value = new Binary(((SqlVarbinary) value).getBytes()); } - - return unmodifiableList(values); } - if (isRowType(type)) { - Block rowBlock = block.getObject(position, Block.class); - - List fieldTypes = type.getTypeParameters(); - if (fieldTypes.size() != rowBlock.getPositionCount()) { - throw new PrestoException(StandardErrorCode.GENERIC_INTERNAL_ERROR, "Expected row value field count does not match type field count"); + else if (type.getJavaType() == Block.class) { + if (isArrayType(type)) { + value = ((List) value).stream() + .map(v -> translateValue(type.getTypeParameters().get(0), v)) + .collect(toList()); } - - if (isImplicitRowType(type)) { - List rowValue = new ArrayList<>(); - for (int i = 0; i < rowBlock.getPositionCount(); i++) { - Object element = getObjectValue(fieldTypes.get(i), rowBlock, i); - rowValue.add(element); + else if (isMapType(type)) { + // map type is converted into list of fixed keys document + ImmutableList.Builder> builder = ImmutableList.builder(); + for (Map.Entry entry : ((Map) value).entrySet()) { + Map mapValue = new HashMap<>(); + mapValue.put("key", translateValue(type.getTypeParameters().get(0), entry.getKey())); + mapValue.put("value", translateValue(type.getTypeParameters().get(1), entry.getValue())); + + builder.add(mapValue); } - return unmodifiableList(rowValue); + value = builder.build(); } - - Map rowValue = new HashMap<>(); - for (int i = 0; i < rowBlock.getPositionCount(); i++) { - rowValue.put( - type.getTypeSignature().getParameters().get(i).getNamedTypeSignature().getName(), - getObjectValue(fieldTypes.get(i), rowBlock, i)); + else if (isRowType(type)) { + List fieldValues = (List) value; + if (isImplicitRowType(type)) { + ArrayList rowValue = new ArrayList<>(); + for (int index = 0; index < fieldValues.size(); index++) { + rowValue.add(translateValue(type.getTypeParameters().get(index), fieldValues.get(index))); + } + value = rowValue; + } + else { + HashMap rowValue = new HashMap<>(); + for (int index = 0; index < fieldValues.size(); index++) { + rowValue.put(type.getTypeSignature().getParameters().get(index).getNamedTypeSignature().getName(), + translateValue(type.getTypeParameters().get(index), fieldValues.get(index))); + } + value = rowValue; + } } - return unmodifiableMap(rowValue); } - throw new PrestoException(NOT_SUPPORTED, "unsupported type: " + type); + return value; } private boolean isImplicitRowType(Type type) diff --git a/presto-mongodb/src/main/java/com/facebook/presto/mongodb/MongoSession.java b/presto-mongodb/src/main/java/com/facebook/presto/mongodb/MongoSession.java index 8aa17f243faf..384ea148b555 100644 --- a/presto-mongodb/src/main/java/com/facebook/presto/mongodb/MongoSession.java +++ b/presto-mongodb/src/main/java/com/facebook/presto/mongodb/MongoSession.java @@ -496,7 +496,7 @@ private List guessTableFields(SchemaTableName schemaTableName) builder.add(metadata); } else { - log.debug("Unable to guess field type from %s : %s", value == null ? "null" : value.getClass().getName(), value); + log.debug("Unable to guess field type from %s : %s", value.getClass().getName(), value); } } diff --git a/presto-mysql/src/test/java/com/facebook/presto/plugin/mysql/TestMySqlDistributedQueries.java b/presto-mysql/src/test/java/com/facebook/presto/plugin/mysql/TestMySqlDistributedQueries.java index 3dd6b683aba2..83e2419626b2 100644 --- a/presto-mysql/src/test/java/com/facebook/presto/plugin/mysql/TestMySqlDistributedQueries.java +++ b/presto-mysql/src/test/java/com/facebook/presto/plugin/mysql/TestMySqlDistributedQueries.java @@ -188,16 +188,16 @@ public void testShowColumns() { MaterializedResult actual = computeActual("SHOW COLUMNS FROM orders"); - MaterializedResult expectedParametrizedVarchar = resultBuilder(getSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) - .row("orderkey", "bigint", "", "") - .row("custkey", "bigint", "", "") - .row("orderstatus", "varchar(255)", "", "") - .row("totalprice", "double", "", "") - .row("orderdate", "date", "", "") - .row("orderpriority", "varchar(255)", "", "") - .row("clerk", "varchar(255)", "", "") - .row("shippriority", "integer", "", "") - .row("comment", "varchar(255)", "", "") + MaterializedResult expectedParametrizedVarchar = resultBuilder(getSession(), VARCHAR, VARCHAR, VARCHAR) + .row("orderkey", "bigint", "") + .row("custkey", "bigint", "") + .row("orderstatus", "varchar(255)", "") + .row("totalprice", "double", "") + .row("orderdate", "date", "") + .row("orderpriority", "varchar(255)", "") + .row("clerk", "varchar(255)", "") + .row("shippriority", "integer", "") + .row("comment", "varchar(255)", "") .build(); assertEquals(actual, expectedParametrizedVarchar); diff --git a/presto-mysql/src/test/java/com/facebook/presto/plugin/mysql/TestMySqlIntegrationSmokeTest.java b/presto-mysql/src/test/java/com/facebook/presto/plugin/mysql/TestMySqlIntegrationSmokeTest.java index 0658f9dd1ee4..c83a0104e40d 100644 --- a/presto-mysql/src/test/java/com/facebook/presto/plugin/mysql/TestMySqlIntegrationSmokeTest.java +++ b/presto-mysql/src/test/java/com/facebook/presto/plugin/mysql/TestMySqlIntegrationSmokeTest.java @@ -27,6 +27,7 @@ import java.sql.Statement; import static com.facebook.presto.plugin.mysql.MySqlQueryRunner.createMySqlQueryRunner; +import static com.facebook.presto.spi.type.TinyintType.TINYINT; import static com.facebook.presto.spi.type.VarcharType.VARCHAR; import static com.facebook.presto.testing.TestingSession.testSessionBuilder; import static com.google.common.collect.Iterables.getOnlyElement; @@ -63,16 +64,16 @@ public void testDescribeTable() MaterializedResult actualColumns = computeActual("DESC ORDERS").toJdbcTypes(); // some connectors don't support dates, and some do not support parametrized varchars, so we check multiple options - MaterializedResult expectedColumns = MaterializedResult.resultBuilder(queryRunner.getDefaultSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) - .row("orderkey", "bigint", "", "") - .row("custkey", "bigint", "", "") - .row("orderstatus", "varchar(255)", "", "") - .row("totalprice", "double", "", "") - .row("orderdate", "date", "", "") - .row("orderpriority", "varchar(255)", "", "") - .row("clerk", "varchar(255)", "", "") - .row("shippriority", "integer", "", "") - .row("comment", "varchar(255)", "", "") + MaterializedResult expectedColumns = MaterializedResult.resultBuilder(queryRunner.getDefaultSession(), VARCHAR, VARCHAR, VARCHAR) + .row("orderkey", "bigint", "") + .row("custkey", "bigint", "") + .row("orderstatus", "varchar(255)", "") + .row("totalprice", "double", "") + .row("orderdate", "date", "") + .row("orderpriority", "varchar(255)", "") + .row("clerk", "varchar(255)", "") + .row("shippriority", "integer", "") + .row("comment", "varchar(255)", "") .build(); assertEquals(actualColumns, expectedColumns); } @@ -120,9 +121,8 @@ public void testMySqlTinyint1() execute("CREATE TABLE tpch.mysql_test_tinyint1 (c_tinyint tinyint(1))"); MaterializedResult actual = computeActual("SHOW COLUMNS FROM mysql_test_tinyint1"); - MaterializedResult expected = MaterializedResult.resultBuilder(getSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) - .row("c_tinyint", "tinyint", "", "") - .build(); + MaterializedResult expected = MaterializedResult.resultBuilder(getSession(), TINYINT) + .row("c_tinyint", "tinyint", "").build(); assertEquals(actual, expected); diff --git a/presto-orc/src/main/java/com/facebook/presto/orc/metadata/DwrfMetadataReader.java b/presto-orc/src/main/java/com/facebook/presto/orc/metadata/DwrfMetadataReader.java index 5f614f122ee0..e71c419b4505 100644 --- a/presto-orc/src/main/java/com/facebook/presto/orc/metadata/DwrfMetadataReader.java +++ b/presto-orc/src/main/java/com/facebook/presto/orc/metadata/DwrfMetadataReader.java @@ -51,7 +51,7 @@ public PostScript readPostScript(byte[] data, int offset, int length) OrcProto.PostScript postScript = OrcProto.PostScript.parseFrom(input); return new PostScript( - ImmutableList.of(), + ImmutableList.of(), postScript.getFooterLength(), 0, toCompression(postScript.getCompression()), @@ -62,7 +62,7 @@ public PostScript readPostScript(byte[] data, int offset, int length) public Metadata readMetadata(InputStream inputStream) throws IOException { - return new Metadata(ImmutableList.of()); + return new Metadata(ImmutableList.of()); } @Override diff --git a/presto-orc/src/main/java/com/facebook/presto/orc/reader/ListStreamReader.java b/presto-orc/src/main/java/com/facebook/presto/orc/reader/ListStreamReader.java index cc1dda1304a0..3f5d815fc7bc 100644 --- a/presto-orc/src/main/java/com/facebook/presto/orc/reader/ListStreamReader.java +++ b/presto-orc/src/main/java/com/facebook/presto/orc/reader/ListStreamReader.java @@ -25,6 +25,7 @@ import com.facebook.presto.spi.block.BlockBuilderStatus; import com.facebook.presto.spi.type.Type; import com.google.common.primitives.Ints; +import io.airlift.slice.Slices; import org.joda.time.DateTimeZone; import javax.annotation.Nonnull; @@ -98,17 +99,13 @@ public Block readBlock(Type type) } } - // The length vector could be reused, but this simplifies the code below by - // taking advantage of null entries being initialized to zero. The vector - // could be reinitialized for each loop, but that is likely just as expensive - // as allocating a new array - int[] lengthVector = new int[nextBatchSize]; + int[] offsets = new int[nextBatchSize]; boolean[] nullVector = new boolean[nextBatchSize]; if (presentStream == null) { if (lengthStream == null) { throw new OrcCorruptionException("Value is not null but data stream is not present"); } - lengthStream.nextIntVector(nextBatchSize, lengthVector); + lengthStream.nextIntVector(nextBatchSize, offsets); } else { int nullValues = presentStream.getUnsetBits(nextBatchSize, nullVector); @@ -116,13 +113,11 @@ public Block readBlock(Type type) if (lengthStream == null) { throw new OrcCorruptionException("Value is not null but data stream is not present"); } - lengthStream.nextIntVector(nextBatchSize, lengthVector, nullVector); + lengthStream.nextIntVector(nextBatchSize, offsets, nullVector); } } - int[] offsets = new int[nextBatchSize + 1]; for (int i = 1; i < offsets.length; i++) { - int length = lengthVector[i - 1]; - offsets[i] = offsets[i - 1] + length; + offsets[i] += offsets[i - 1]; } Type elementType = type.getTypeParameters().get(0); @@ -136,7 +131,7 @@ public Block readBlock(Type type) else { elements = elementType.createBlockBuilder(new BlockBuilderStatus(), 0).build(); } - ArrayBlock arrayBlock = new ArrayBlock(nextBatchSize, nullVector, offsets, elements); + ArrayBlock arrayBlock = new ArrayBlock(elements, Slices.wrappedIntArray(offsets), 0, Slices.wrappedBooleanArray(nullVector)); readOffset = 0; nextBatchSize = 0; diff --git a/presto-orc/src/main/java/com/facebook/presto/orc/reader/MapStreamReader.java b/presto-orc/src/main/java/com/facebook/presto/orc/reader/MapStreamReader.java index 625b0efcd44d..0abdbe1dbec2 100644 --- a/presto-orc/src/main/java/com/facebook/presto/orc/reader/MapStreamReader.java +++ b/presto-orc/src/main/java/com/facebook/presto/orc/reader/MapStreamReader.java @@ -26,6 +26,7 @@ import com.facebook.presto.spi.block.InterleavedBlock; import com.facebook.presto.spi.type.Type; import com.google.common.primitives.Ints; +import io.airlift.slice.Slices; import it.unimi.dsi.fastutil.ints.IntArrayList; import org.joda.time.DateTimeZone; @@ -103,17 +104,13 @@ public Block readBlock(Type type) } } - // The length vector could be reused, but this simplifies the code below by - // taking advantage of null entries being initialized to zero. The vector - // could be reinitialized for each loop, but that is likely just as expensive - // as allocating a new array - int[] lengthVector = new int[nextBatchSize]; + int[] lengths = new int[nextBatchSize]; boolean[] nullVector = new boolean[nextBatchSize]; if (presentStream == null) { if (lengthStream == null) { throw new OrcCorruptionException("Value is not null but data stream is not present"); } - lengthStream.nextIntVector(nextBatchSize, lengthVector); + lengthStream.nextIntVector(nextBatchSize, lengths); } else { int nullValues = presentStream.getUnsetBits(nextBatchSize, nullVector); @@ -121,7 +118,7 @@ public Block readBlock(Type type) if (lengthStream == null) { throw new OrcCorruptionException("Value is not null but data stream is not present"); } - lengthStream.nextIntVector(nextBatchSize, lengthVector, nullVector); + lengthStream.nextIntVector(nextBatchSize, lengths, nullVector); } } @@ -129,7 +126,7 @@ public Block readBlock(Type type) Type valueType = type.getTypeParameters().get(1); int entryCount = 0; - for (int length : lengthVector) { + for (int length : lengths) { entryCount += length; } @@ -146,15 +143,14 @@ public Block readBlock(Type type) values = valueType.createBlockBuilder(new BlockBuilderStatus(), 1).build(); } - InterleavedBlock keyValueBlock = createKeyValueBlock(nextBatchSize, keys, values, lengthVector); + InterleavedBlock keyValueBlock = createKeyValueBlock(keys, values, lengths); // convert lengths into offsets into the keyValueBlock (e.g., two positions per entry) - int[] offsets = new int[nextBatchSize + 1]; - for (int i = 1; i < offsets.length; i++) { - int length = lengthVector[i - 1] * 2; - offsets[i] = offsets[i - 1] + length; + lengths[0] = lengths[0] * 2; + for (int i = 1; i < lengths.length; i++) { + lengths[i] = lengths[i - 1] + (lengths[i] * 2); } - ArrayBlock arrayBlock = new ArrayBlock(nextBatchSize, nullVector, offsets, keyValueBlock); + ArrayBlock arrayBlock = new ArrayBlock(keyValueBlock, Slices.wrappedIntArray(lengths), 0, Slices.wrappedBooleanArray(nullVector)); readOffset = 0; nextBatchSize = 0; @@ -162,7 +158,7 @@ public Block readBlock(Type type) return arrayBlock; } - private static InterleavedBlock createKeyValueBlock(int positionCount, Block keys, Block values, int[] lengths) + private static InterleavedBlock createKeyValueBlock(Block keys, Block values, int[] lengths) { if (!hasNull(keys)) { return new InterleavedBlock(new Block[] {keys, values}); @@ -175,7 +171,7 @@ private static InterleavedBlock createKeyValueBlock(int positionCount, Block key IntArrayList nonNullPositions = new IntArrayList(keys.getPositionCount()); int position = 0; - for (int mapIndex = 0; mapIndex < positionCount; mapIndex++) { + for (int mapIndex = 0; mapIndex < lengths.length; mapIndex++) { int length = lengths[mapIndex]; for (int entryIndex = 0; entryIndex < length; entryIndex++) { if (keys.isNull(position)) { diff --git a/presto-orc/src/main/java/com/facebook/presto/orc/reader/StructStreamReader.java b/presto-orc/src/main/java/com/facebook/presto/orc/reader/StructStreamReader.java index 7d5620b2473c..1d04c8accc12 100644 --- a/presto-orc/src/main/java/com/facebook/presto/orc/reader/StructStreamReader.java +++ b/presto-orc/src/main/java/com/facebook/presto/orc/reader/StructStreamReader.java @@ -23,6 +23,7 @@ import com.facebook.presto.spi.block.BlockBuilderStatus; import com.facebook.presto.spi.block.InterleavedBlock; import com.facebook.presto.spi.type.Type; +import io.airlift.slice.Slices; import org.joda.time.DateTimeZone; import javax.annotation.Nonnull; @@ -120,15 +121,15 @@ public Block readBlock(Type type) } // Build offsets for array block (null valued have no positions) - int[] offsets = new int[nextBatchSize + 1]; - for (int i = 1; i < offsets.length; i++) { - int length = nullVector[i - 1] ? 0 : typeParameters.size(); - offsets[i] = offsets[i - 1] + length; + int[] offsets = new int[nextBatchSize]; + offsets[0] = (nullVector[0] ? 0 : typeParameters.size()); + for (int i = 1; i < nextBatchSize; i++) { + offsets[i] = offsets[i - 1] + (nullVector[i] ? 0 : typeParameters.size()); } // Struct is represented as an array block holding an interleaved block InterleavedBlock interleavedBlock = new InterleavedBlock(blocks); - ArrayBlock arrayBlock = new ArrayBlock(nextBatchSize, nullVector, offsets, interleavedBlock); + ArrayBlock arrayBlock = new ArrayBlock(interleavedBlock, Slices.wrappedIntArray(offsets), 0, Slices.wrappedBooleanArray(nullVector)); readOffset = 0; nextBatchSize = 0; diff --git a/presto-orc/src/test/java/com/facebook/presto/orc/TestTupleDomainOrcPredicate.java b/presto-orc/src/test/java/com/facebook/presto/orc/TestTupleDomainOrcPredicate.java index 3f685c91150d..a979a52bbfe1 100644 --- a/presto-orc/src/test/java/com/facebook/presto/orc/TestTupleDomainOrcPredicate.java +++ b/presto-orc/src/test/java/com/facebook/presto/orc/TestTupleDomainOrcPredicate.java @@ -232,8 +232,6 @@ public void testChar() assertEquals(getDomain(CHAR, 10, stringColumnStats(5L, null, "taco")), create(ValueSet.ofRanges(lessThanOrEqual(CHAR, utf8Slice("taco"))), true)); assertEquals(getDomain(CHAR, 10, stringColumnStats(5L, "apple ", null)), create(ValueSet.ofRanges(greaterThanOrEqual(CHAR, utf8Slice("apple"))), true)); assertEquals(getDomain(CHAR, 10, stringColumnStats(5L, "apple", null)), create(ValueSet.ofRanges(greaterThanOrEqual(CHAR, utf8Slice("apple"))), true)); - - assertEquals(getDomain(CHAR, 10, stringColumnStats(10L, "\0 ", " ")), create(ValueSet.ofRanges(range(CHAR, utf8Slice("\0"), true, utf8Slice(""), true)), false)); } private static ColumnStatistics stringColumnStats(Long numberOfValues, String minimum, String maximum) diff --git a/presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4 b/presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4 index e2b774df0ade..df2114fc5dcb 100644 --- a/presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4 +++ b/presto-parser/src/main/antlr4/com/facebook/presto/sql/parser/SqlBase.g4 @@ -280,10 +280,7 @@ primaryExpression | BINARY_LITERAL #binaryLiteral | '?' #parameter | POSITION '(' valueExpression IN valueExpression ')' #position - // This case handles both an implicit row constructor or a simple parenthesized - // expression. We can't make the two separate alternatives because it needs - // unbounded look-ahead to figure out which one to take while it looks for the comma - | '(' expression (',' expression)* ')' #implicitRowConstructor + | '(' expression (',' expression)+ ')' #rowConstructor | ROW '(' expression (',' expression)* ')' #rowConstructor | qualifiedName '(' ASTERISK ')' filter? over? #functionCall | qualifiedName '(' (setQuantifier? expression (',' expression)*)? ')' filter? over? #functionCall @@ -308,6 +305,7 @@ primaryExpression | SUBSTRING '(' valueExpression FROM valueExpression (FOR valueExpression)? ')' #substring | NORMALIZE '(' valueExpression (',' normalForm)? ')' #normalize | EXTRACT '(' identifier FROM valueExpression ')' #extract + | '(' expression ')' #parenthesizedExpression ; timeZoneSpecifier diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/ExpressionFormatter.java b/presto-parser/src/main/java/com/facebook/presto/sql/ExpressionFormatter.java index 5e3dbfad7622..bc877d41c4a1 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/ExpressionFormatter.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/ExpressionFormatter.java @@ -46,7 +46,6 @@ import com.facebook.presto.sql.tree.IntervalLiteral; import com.facebook.presto.sql.tree.IsNotNullPredicate; import com.facebook.presto.sql.tree.IsNullPredicate; -import com.facebook.presto.sql.tree.LambdaArgumentDeclaration; import com.facebook.presto.sql.tree.LambdaExpression; import com.facebook.presto.sql.tree.LikePredicate; import com.facebook.presto.sql.tree.LogicalBinaryExpression; @@ -292,12 +291,6 @@ protected String visitQualifiedNameReference(QualifiedNameReference node, Boolea return formatQualifiedName(node.getName()); } - @Override - protected String visitLambdaArgumentDeclaration(LambdaArgumentDeclaration node, Boolean context) - { - return formatIdentifier(node.getName()); - } - @Override protected String visitSymbolReference(SymbolReference node, Boolean context) { diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/SqlFormatter.java b/presto-parser/src/main/java/com/facebook/presto/sql/SqlFormatter.java index 052003362110..a614ffce8870 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/SqlFormatter.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/SqlFormatter.java @@ -341,8 +341,7 @@ protected Void visitAllColumns(AllColumns node, Integer context) @Override protected Void visitTable(Table node, Integer indent) { - builder.append(formatName(node.getName())); - + builder.append(node.getName().toString()); return null; } diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/parser/AstBuilder.java b/presto-parser/src/main/java/com/facebook/presto/sql/parser/AstBuilder.java index 4d6840389d4e..f8b680427186 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/parser/AstBuilder.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/parser/AstBuilder.java @@ -79,7 +79,6 @@ import com.facebook.presto.sql.tree.JoinCriteria; import com.facebook.presto.sql.tree.JoinOn; import com.facebook.presto.sql.tree.JoinUsing; -import com.facebook.presto.sql.tree.LambdaArgumentDeclaration; import com.facebook.presto.sql.tree.LambdaExpression; import com.facebook.presto.sql.tree.LikeClause; import com.facebook.presto.sql.tree.LikePredicate; @@ -447,7 +446,7 @@ public Node visitQueryNoWith(SqlBaseParser.QueryNoWithContext context) return new Query( getLocation(context), - Optional.empty(), + Optional.empty(), new QuerySpecification( getLocation(context), query.getSelect(), @@ -458,12 +457,12 @@ public Node visitQueryNoWith(SqlBaseParser.QueryNoWithContext context) visit(context.sortItem(), SortItem.class), getTextIfPresent(context.limit)), ImmutableList.of(), - Optional.empty()); + Optional.empty()); } return new Query( getLocation(context), - Optional.empty(), + Optional.empty(), term, visit(context.sortItem(), SortItem.class), getTextIfPresent(context.limit)); @@ -482,7 +481,7 @@ public Node visitQuerySpecification(SqlBaseParser.QuerySpecificationContext cont Relation relation = iterator.next(); while (iterator.hasNext()) { - relation = new Join(getLocation(context), Join.Type.IMPLICIT, relation, iterator.next(), Optional.empty()); + relation = new Join(getLocation(context), Join.Type.IMPLICIT, relation, iterator.next(), Optional.empty()); } from = Optional.of(relation); @@ -773,7 +772,7 @@ public Node visitJoinRelation(SqlBaseParser.JoinRelationContext context) if (context.CROSS() != null) { right = (Relation) visit(context.right); - return new Join(getLocation(context), Join.Type.CROSS, left, right, Optional.empty()); + return new Join(getLocation(context), Join.Type.CROSS, left, right, Optional.empty()); } JoinCriteria criteria; @@ -1059,14 +1058,9 @@ public Node visitTimeZoneString(SqlBaseParser.TimeZoneStringContext context) // ********************* primary expressions ********************** @Override - public Node visitImplicitRowConstructor(SqlBaseParser.ImplicitRowConstructorContext context) + public Node visitParenthesizedExpression(SqlBaseParser.ParenthesizedExpressionContext context) { - List items = visit(context.expression(), Expression.class); - if (items.size() == 1) { - return items.get(0); - } - - return new Row(getLocation(context), items); + return visit(context.expression()); } @Override @@ -1246,9 +1240,8 @@ public Node visitFunctionCall(SqlBaseParser.FunctionCallContext context) @Override public Node visitLambda(SqlBaseParser.LambdaContext context) { - List arguments = context.identifier().stream() + List arguments = context.identifier().stream() .map(SqlBaseParser.IdentifierContext::getText) - .map(LambdaArgumentDeclaration::new) .collect(toList()); Expression body = (Expression) visit(context.expression()); diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/parser/StatementSplitter.java b/presto-parser/src/main/java/com/facebook/presto/sql/parser/StatementSplitter.java index 799459ba1937..491891bd30f6 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/parser/StatementSplitter.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/parser/StatementSplitter.java @@ -73,7 +73,7 @@ public String getPartialStatement() public static String squeezeStatement(String sql) { - TokenSource tokens = getLexer(sql, ImmutableSet.of()); + TokenSource tokens = getLexer(sql, ImmutableSet.of()); StringBuilder sb = new StringBuilder(); while (true) { Token token = tokens.nextToken(); @@ -92,7 +92,7 @@ public static String squeezeStatement(String sql) public static boolean isEmptyStatement(String sql) { - TokenSource tokens = getLexer(sql, ImmutableSet.of()); + TokenSource tokens = getLexer(sql, ImmutableSet.of()); while (true) { Token token = tokens.nextToken(); if (token.getType() == Token.EOF) { diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/AddColumn.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/AddColumn.java index 6f57aafef88b..a06d1f5d2550 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/AddColumn.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/AddColumn.java @@ -20,7 +20,7 @@ import static java.util.Objects.requireNonNull; public class AddColumn - extends Statement + extends DataDefinitionStatement { private final QualifiedName name; private final ColumnDefinition column; diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/AstVisitor.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/AstVisitor.java index dff92e76cba8..ff11b8ba149f 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/AstVisitor.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/AstVisitor.java @@ -17,11 +17,6 @@ public abstract class AstVisitor { - public R process(Node node) - { - return process(node, null); - } - public R process(Node node, @Nullable C context) { return node.accept(this, context); @@ -87,14 +82,19 @@ protected R visitStatement(Statement node, C context) return visitNode(node, context); } - protected R visitPrepare(Prepare node, C context) + protected R visitDataDefinitionStatement(DataDefinitionStatement node, C context) { return visitStatement(node, context); } + protected R visitPrepare(Prepare node, C context) + { + return visitDataDefinitionStatement(node, context); + } + protected R visitDeallocate(Deallocate node, C context) { - return visitStatement(node, context); + return visitDataDefinitionStatement(node, context); } protected R visitExecute(Execute node, C context) @@ -169,12 +169,12 @@ protected R visitShowSession(ShowSession node, C context) protected R visitSetSession(SetSession node, C context) { - return visitStatement(node, context); + return visitDataDefinitionStatement(node, context); } - protected R visitResetSession(ResetSession node, C context) + public R visitResetSession(ResetSession node, C context) { - return visitStatement(node, context); + return visitDataDefinitionStatement(node, context); } protected R visitGenericLiteral(GenericLiteral node, C context) @@ -519,7 +519,7 @@ protected R visitRenameSchema(RenameSchema node, C context) protected R visitCreateTable(CreateTable node, C context) { - return visitStatement(node, context); + return visitDataDefinitionStatement(node, context); } protected R visitCreateTableAsSelect(CreateTableAsSelect node, C context) @@ -529,32 +529,32 @@ protected R visitCreateTableAsSelect(CreateTableAsSelect node, C context) protected R visitDropTable(DropTable node, C context) { - return visitStatement(node, context); + return visitDataDefinitionStatement(node, context); } protected R visitRenameTable(RenameTable node, C context) { - return visitStatement(node, context); + return visitDataDefinitionStatement(node, context); } protected R visitRenameColumn(RenameColumn node, C context) { - return visitStatement(node, context); + return visitDataDefinitionStatement(node, context); } protected R visitAddColumn(AddColumn node, C context) { - return visitStatement(node, context); + return visitDataDefinitionStatement(node, context); } protected R visitCreateView(CreateView node, C context) { - return visitStatement(node, context); + return visitDataDefinitionStatement(node, context); } protected R visitDropView(DropView node, C context) { - return visitStatement(node, context); + return visitDataDefinitionStatement(node, context); } protected R visitInsert(Insert node, C context) @@ -564,27 +564,27 @@ protected R visitInsert(Insert node, C context) protected R visitCall(Call node, C context) { - return visitStatement(node, context); + return visitDataDefinitionStatement(node, context); } protected R visitDelete(Delete node, C context) { - return visitStatement(node, context); + return visitDataDefinitionStatement(node, context); } protected R visitStartTransaction(StartTransaction node, C context) { - return visitStatement(node, context); + return visitDataDefinitionStatement(node, context); } protected R visitGrant(Grant node, C context) { - return visitStatement(node, context); + return visitDataDefinitionStatement(node, context); } protected R visitRevoke(Revoke node, C context) { - return visitStatement(node, context); + return visitDataDefinitionStatement(node, context); } protected R visitTransactionMode(TransactionMode node, C context) @@ -604,12 +604,12 @@ protected R visitTransactionAccessMode(TransactionAccessMode node, C context) protected R visitCommit(Commit node, C context) { - return visitStatement(node, context); + return visitDataDefinitionStatement(node, context); } protected R visitRollback(Rollback node, C context) { - return visitStatement(node, context); + return visitDataDefinitionStatement(node, context); } protected R visitAtTimeZone(AtTimeZone node, C context) @@ -656,9 +656,4 @@ protected R visitQuantifiedComparisonExpression(QuantifiedComparisonExpression n { return visitExpression(node, context); } - - protected R visitLambdaArgumentDeclaration(LambdaArgumentDeclaration node, C context) - { - return visitExpression(node, context); - } } diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/Call.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/Call.java index 81c387d84ef9..3bbf7252a459 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/Call.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/Call.java @@ -23,7 +23,7 @@ import static java.util.Objects.requireNonNull; public final class Call - extends Statement + extends DataDefinitionStatement { private final QualifiedName name; private final List arguments; diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/Cast.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/Cast.java index b3fcac2a7df6..4a2a21e12d93 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/Cast.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/Cast.java @@ -64,7 +64,7 @@ private Cast(Optional location, Expression expression, String type requireNonNull(type, "type is null"); this.expression = expression; - this.type = type.toLowerCase(ENGLISH); + this.type = type.toUpperCase(ENGLISH); this.safe = safe; this.typeOnly = typeOnly; } diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/Commit.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/Commit.java index 4ab127653962..8c8212b7fcdb 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/Commit.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/Commit.java @@ -18,7 +18,7 @@ import java.util.Optional; public final class Commit - extends Statement + extends DataDefinitionStatement { public Commit() { diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/CreateTable.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/CreateTable.java index 472650cab4df..4166e37458e3 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/CreateTable.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/CreateTable.java @@ -25,7 +25,7 @@ import static java.util.Objects.requireNonNull; public class CreateTable - extends Statement + extends DataDefinitionStatement { private final QualifiedName name; private final List elements; diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/CreateView.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/CreateView.java index 07e50b6be2b0..74af4135a3af 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/CreateView.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/CreateView.java @@ -20,7 +20,7 @@ import static java.util.Objects.requireNonNull; public class CreateView - extends Statement + extends DataDefinitionStatement { private final QualifiedName name; private final Query query; diff --git a/presto-main/src/main/java/com/facebook/presto/spiller/SpillerFactoryWithStats.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/DataDefinitionStatement.java similarity index 61% rename from presto-main/src/main/java/com/facebook/presto/spiller/SpillerFactoryWithStats.java rename to presto-parser/src/main/java/com/facebook/presto/sql/tree/DataDefinitionStatement.java index 21abd0b39e7e..82e60d647d78 100644 --- a/presto-main/src/main/java/com/facebook/presto/spiller/SpillerFactoryWithStats.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/DataDefinitionStatement.java @@ -11,21 +11,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.facebook.presto.spiller; -import org.weakref.jmx.Managed; +package com.facebook.presto.sql.tree; -import java.util.concurrent.atomic.AtomicLong; +import java.util.Optional; -public abstract class SpillerFactoryWithStats - implements SpillerFactory +public abstract class DataDefinitionStatement + extends Statement { - protected final AtomicLong spilledBytes = new AtomicLong(); + protected DataDefinitionStatement(Optional location) + { + super(location); + } @Override - @Managed - public long getSpilledBytes() + public R accept(AstVisitor visitor, C context) { - return spilledBytes.get(); + return visitor.visitDataDefinitionStatement(this, context); } } diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/Deallocate.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/Deallocate.java index 2a9a9cff9a38..a8705637b08f 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/Deallocate.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/Deallocate.java @@ -20,7 +20,7 @@ import static java.util.Objects.requireNonNull; public class Deallocate - extends Statement + extends DataDefinitionStatement { private final String name; diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/Delete.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/Delete.java index f8d4a8834b59..439398949a9d 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/Delete.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/Delete.java @@ -20,7 +20,7 @@ import static java.util.Objects.requireNonNull; public class Delete - extends Statement + extends DataDefinitionStatement { private final Table table; private final Optional where; diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/DropTable.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/DropTable.java index a5a2b71f1a30..d604f1458bea 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/DropTable.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/DropTable.java @@ -19,7 +19,7 @@ import static com.google.common.base.MoreObjects.toStringHelper; public class DropTable - extends Statement + extends DataDefinitionStatement { private final QualifiedName tableName; private final boolean exists; diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/DropView.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/DropView.java index 4450a1207b75..a825de6727e3 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/DropView.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/DropView.java @@ -19,7 +19,7 @@ import static com.google.common.base.MoreObjects.toStringHelper; public class DropView - extends Statement + extends DataDefinitionStatement { private final QualifiedName name; private final boolean exists; diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/ExpressionTreeRewriter.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/ExpressionTreeRewriter.java index 6697d0b32304..40dbbbb9b13e 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/ExpressionTreeRewriter.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/ExpressionTreeRewriter.java @@ -515,7 +515,7 @@ public Expression visitFunctionCall(FunctionCall node, Context context) if (start.getValue().isPresent()) { Expression value = rewrite(start.getValue().get(), context.get()); if (value != start.getValue().get()) { - start = new FrameBound(start.getType(), value, start.getOriginalValue().get()); + start = new FrameBound(start.getType(), value); } } @@ -525,8 +525,7 @@ public Expression visitFunctionCall(FunctionCall node, Context context) if (value.isPresent()) { Expression rewrittenValue = rewrite(value.get(), context.get()); if (rewrittenValue != value.get()) { - rewrittenEnd = Optional.of(new FrameBound(rewrittenEnd.get().getType(), - rewrittenValue, rewrittenEnd.get().getOriginalValue().get())); + rewrittenEnd = Optional.of(new FrameBound(rewrittenEnd.get().getType(), rewrittenValue)); } } } diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/FrameBound.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/FrameBound.java index 5216619723ec..45be61cc6421 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/FrameBound.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/FrameBound.java @@ -33,7 +33,6 @@ public enum Type private final Type type; private final Optional value; - private final Optional originalValue; public FrameBound(Type type) { @@ -45,27 +44,26 @@ public FrameBound(NodeLocation location, Type type) this(Optional.of(location), type); } - public FrameBound(Type type, Expression value, Expression originalValue) + public FrameBound(Type type, Expression value) { - this(Optional.empty(), type, value, originalValue); + this(Optional.empty(), type, value); } private FrameBound(Optional location, Type type) { - this(location, type, null, null); + this(location, type, null); } public FrameBound(NodeLocation location, Type type, Expression value) { - this(Optional.of(location), type, value, value); + this(Optional.of(location), type, value); } - private FrameBound(Optional location, Type type, Expression value, Expression originalValue) + private FrameBound(Optional location, Type type, Expression value) { super(location); this.type = requireNonNull(type, "type is null"); this.value = Optional.ofNullable(value); - this.originalValue = Optional.ofNullable(originalValue); } public Type getType() @@ -78,11 +76,6 @@ public Optional getValue() return value; } - public Optional getOriginalValue() - { - return originalValue; - } - @Override public R accept(AstVisitor visitor, C context) { diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/Grant.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/Grant.java index da061366bf07..9714a3f16e1b 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/Grant.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/Grant.java @@ -23,7 +23,7 @@ import static java.util.Objects.requireNonNull; public class Grant - extends Statement + extends DataDefinitionStatement { private final Optional> privileges; // missing means ALL PRIVILEGES private final boolean table; diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/LambdaArgumentDeclaration.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/LambdaArgumentDeclaration.java deleted file mode 100644 index 9cb427395d56..000000000000 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/LambdaArgumentDeclaration.java +++ /dev/null @@ -1,60 +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.sql.tree; - -import java.util.Objects; -import java.util.Optional; - -public class LambdaArgumentDeclaration - extends Expression -{ - private final String name; - - public LambdaArgumentDeclaration(String name) - { - super(Optional.empty()); - this.name = name; - } - - public String getName() - { - return name; - } - - @Override - public R accept(AstVisitor visitor, C context) - { - return visitor.visitLambdaArgumentDeclaration(this, context); - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - LambdaArgumentDeclaration that = (LambdaArgumentDeclaration) o; - return Objects.equals(name, that.name); - } - - @Override - public int hashCode() - { - return Objects.hash(name); - } -} diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/LambdaExpression.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/LambdaExpression.java index bd01c20c587d..889d0d0a7836 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/LambdaExpression.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/LambdaExpression.java @@ -22,27 +22,27 @@ public class LambdaExpression extends Expression { - private final List arguments; + private final List arguments; private final Expression body; - public LambdaExpression(List arguments, Expression body) + public LambdaExpression(List arguments, Expression body) { this(Optional.empty(), arguments, body); } - public LambdaExpression(NodeLocation location, List arguments, Expression body) + public LambdaExpression(NodeLocation location, List arguments, Expression body) { this(Optional.of(location), arguments, body); } - private LambdaExpression(Optional location, List arguments, Expression body) + private LambdaExpression(Optional location, List arguments, Expression body) { super(location); this.arguments = requireNonNull(arguments, "arguments is null"); this.body = requireNonNull(body, "body is null"); } - public List getArguments() + public List getArguments() { return arguments; } diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/Prepare.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/Prepare.java index 2d7ce4d105b8..b9da5ee32a82 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/Prepare.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/Prepare.java @@ -20,7 +20,7 @@ import static java.util.Objects.requireNonNull; public class Prepare - extends Statement + extends DataDefinitionStatement { private final String name; private final Statement statement; diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/RenameColumn.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/RenameColumn.java index 6bbfe8e37a68..ab82083dacb4 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/RenameColumn.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/RenameColumn.java @@ -20,7 +20,7 @@ import static java.util.Objects.requireNonNull; public class RenameColumn - extends Statement + extends DataDefinitionStatement { private final QualifiedName table; private final String source; diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/RenameTable.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/RenameTable.java index 3eede3c6dcee..fc5ccb048369 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/RenameTable.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/RenameTable.java @@ -20,7 +20,7 @@ import static java.util.Objects.requireNonNull; public final class RenameTable - extends Statement + extends DataDefinitionStatement { private final QualifiedName source; private final QualifiedName target; diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/ResetSession.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/ResetSession.java index d073f1976d40..8476d8b24274 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/ResetSession.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/ResetSession.java @@ -19,7 +19,7 @@ import static com.google.common.base.MoreObjects.toStringHelper; public class ResetSession - extends Statement + extends DataDefinitionStatement { private final QualifiedName name; diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/Revoke.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/Revoke.java index e9b339d2f439..873ac3cac604 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/Revoke.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/Revoke.java @@ -23,7 +23,7 @@ import static java.util.Objects.requireNonNull; public class Revoke - extends Statement + extends DataDefinitionStatement { private final boolean grantOptionFor; private final Optional> privileges; // missing means ALL PRIVILEGES diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/Rollback.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/Rollback.java index e50380de5357..7f542b9f8610 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/Rollback.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/Rollback.java @@ -17,7 +17,7 @@ import java.util.Optional; public final class Rollback - extends Statement + extends DataDefinitionStatement { public Rollback() { diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/SetSession.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/SetSession.java index 75999b680f13..dd5e02eb7faa 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/SetSession.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/SetSession.java @@ -19,7 +19,7 @@ import static com.google.common.base.MoreObjects.toStringHelper; public class SetSession - extends Statement + extends DataDefinitionStatement { private final QualifiedName name; private final Expression value; diff --git a/presto-parser/src/main/java/com/facebook/presto/sql/tree/StartTransaction.java b/presto-parser/src/main/java/com/facebook/presto/sql/tree/StartTransaction.java index 1ecd3ef166d1..3eea4a8e35e5 100644 --- a/presto-parser/src/main/java/com/facebook/presto/sql/tree/StartTransaction.java +++ b/presto-parser/src/main/java/com/facebook/presto/sql/tree/StartTransaction.java @@ -24,7 +24,7 @@ import static java.util.Objects.requireNonNull; public final class StartTransaction - extends Statement + extends DataDefinitionStatement { private final List transactionModes; diff --git a/presto-parser/src/test/java/com/facebook/presto/sql/parser/TestSqlParser.java b/presto-parser/src/test/java/com/facebook/presto/sql/parser/TestSqlParser.java index 5f79d2f2652d..cacdaf1e523e 100644 --- a/presto-parser/src/test/java/com/facebook/presto/sql/parser/TestSqlParser.java +++ b/presto-parser/src/test/java/com/facebook/presto/sql/parser/TestSqlParser.java @@ -63,8 +63,8 @@ import com.facebook.presto.sql.tree.IntervalLiteral.Sign; import com.facebook.presto.sql.tree.Isolation; import com.facebook.presto.sql.tree.Join; +import com.facebook.presto.sql.tree.JoinCriteria; import com.facebook.presto.sql.tree.JoinOn; -import com.facebook.presto.sql.tree.LambdaArgumentDeclaration; import com.facebook.presto.sql.tree.LambdaExpression; import com.facebook.presto.sql.tree.LikeClause; import com.facebook.presto.sql.tree.LogicalBinaryExpression; @@ -166,26 +166,6 @@ public void testPossibleExponentialBacktracking() SQL_PARSER.createExpression("(((((((((((((((((((((((((((true)))))))))))))))))))))))))))"); } - @Test(timeOut = 2_000) - public void testPotentialUnboundedLookahead() - throws Exception - { - SQL_PARSER.createExpression("(\n" + - " 1 * -1 +\n" + - " 1 * -2 +\n" + - " 1 * -3 +\n" + - " 1 * -4 +\n" + - " 1 * -5 +\n" + - " 1 * -6 +\n" + - " 1 * -7 +\n" + - " 1 * -8 +\n" + - " 1 * -9 +\n" + - " 1 * -10 +\n" + - " 1 * -11 +\n" + - " 1 * -12 \n" + - ")\n"); - } - @Test public void testQualifiedName() { @@ -246,11 +226,11 @@ public void testLiterals() public void testArrayConstructor() throws Exception { - assertExpression("ARRAY []", new ArrayConstructor(ImmutableList.of())); - assertExpression("ARRAY [1, 2]", new ArrayConstructor(ImmutableList.of(new LongLiteral("1"), new LongLiteral("2")))); - assertExpression("ARRAY [1.0, 2.5]", new ArrayConstructor(ImmutableList.of(new DoubleLiteral("1.0"), new DoubleLiteral("2.5")))); - assertExpression("ARRAY ['hi']", new ArrayConstructor(ImmutableList.of(new StringLiteral("hi")))); - assertExpression("ARRAY ['hi', 'hello']", new ArrayConstructor(ImmutableList.of(new StringLiteral("hi"), new StringLiteral("hello")))); + assertExpression("ARRAY []", new ArrayConstructor(ImmutableList.of())); + assertExpression("ARRAY [1, 2]", new ArrayConstructor(ImmutableList.of(new LongLiteral("1"), new LongLiteral("2")))); + assertExpression("ARRAY [1.0, 2.5]", new ArrayConstructor(ImmutableList.of(new DoubleLiteral("1.0"), new DoubleLiteral("2.5")))); + assertExpression("ARRAY ['hi']", new ArrayConstructor(ImmutableList.of(new StringLiteral("hi")))); + assertExpression("ARRAY ['hi', 'hello']", new ArrayConstructor(ImmutableList.of(new StringLiteral("hi"), new StringLiteral("hello")))); } @Test @@ -258,7 +238,7 @@ public void testArraySubscript() throws Exception { assertExpression("ARRAY [1, 2][1]", new SubscriptExpression( - new ArrayConstructor(ImmutableList.of(new LongLiteral("1"), new LongLiteral("2"))), + new ArrayConstructor(ImmutableList.of(new LongLiteral("1"), new LongLiteral("2"))), new LongLiteral("1")) ); try { @@ -424,7 +404,7 @@ public void testUnion() new Union(ImmutableList.of(createSelect123(), createSelect123()), true), createSelect123() ), false), - ImmutableList.of(), + ImmutableList.of(), Optional.empty())); } @@ -793,7 +773,7 @@ public void testShowCatalogs() public void testShowSchemas() throws Exception { - assertStatement("SHOW SCHEMAS", new ShowSchemas(Optional.empty(), Optional.empty())); + assertStatement("SHOW SCHEMAS", new ShowSchemas(Optional.empty(), Optional.empty())); assertStatement("SHOW SCHEMAS FROM foo", new ShowSchemas(Optional.of("foo"), Optional.empty())); assertStatement("SHOW SCHEMAS IN foo LIKE '%'", new ShowSchemas(Optional.of("foo"), Optional.of("%"))); } @@ -856,7 +836,7 @@ public void testSubstringBuiltInFunction() Optional.empty(), ImmutableList.of(), Optional.empty()), - ImmutableList.of(), + ImmutableList.of(), Optional.empty())); assertStatement(format("SELECT substring('%s' FROM 2 FOR 3)", givenString), @@ -870,7 +850,7 @@ public void testSubstringBuiltInFunction() Optional.empty(), ImmutableList.of(), Optional.empty()), - ImmutableList.of(), + ImmutableList.of(), Optional.empty())); } @@ -889,7 +869,7 @@ public void testSubstringRegisteredFunction() Optional.empty(), ImmutableList.of(), Optional.empty()), - ImmutableList.of(), + ImmutableList.of(), Optional.empty())); assertStatement(format("SELECT substring('%s', 2, 3)", givenString), @@ -903,7 +883,7 @@ public void testSubstringRegisteredFunction() Optional.empty(), ImmutableList.of(), Optional.empty()), - ImmutableList.of(), + ImmutableList.of(), Optional.empty())); } @@ -926,7 +906,7 @@ public void testSelectWithRowType() Optional.empty(), ImmutableList.of(), Optional.empty()), - ImmutableList.of(), + ImmutableList.of(), Optional.empty())); assertStatement("SELECT col1.f1[0], col2, col3[2].f2.f3, col4[4] FROM table1", @@ -945,7 +925,7 @@ public void testSelectWithRowType() Optional.empty(), ImmutableList.of(), Optional.empty()), - ImmutableList.of(), + ImmutableList.of(), Optional.empty())); assertStatement("SELECT CAST(ROW(11, 12) AS ROW(COL0 INTEGER, COL1 INTEGER)).col0", @@ -961,7 +941,7 @@ public void testSelectWithRowType() Optional.empty(), ImmutableList.of(), Optional.empty()), - ImmutableList.of(), + ImmutableList.of(), Optional.empty())); } @@ -980,7 +960,7 @@ public void testSelectWithGroupBy() Optional.empty(), ImmutableList.of(), Optional.empty()), - ImmutableList.of(), + ImmutableList.of(), Optional.empty())); assertStatement("SELECT * FROM table1 GROUP BY a, b", @@ -996,7 +976,7 @@ public void testSelectWithGroupBy() Optional.empty(), ImmutableList.of(), Optional.empty()), - ImmutableList.of(), + ImmutableList.of(), Optional.empty())); assertStatement("SELECT * FROM table1 GROUP BY ()", @@ -1010,7 +990,7 @@ public void testSelectWithGroupBy() Optional.empty(), ImmutableList.of(), Optional.empty()), - ImmutableList.of(), + ImmutableList.of(), Optional.empty())); assertStatement("SELECT * FROM table1 GROUP BY GROUPING SETS (a)", @@ -1024,7 +1004,7 @@ public void testSelectWithGroupBy() Optional.empty(), ImmutableList.of(), Optional.empty()), - ImmutableList.of(), + ImmutableList.of(), Optional.empty())); assertStatement("SELECT * FROM table1 GROUP BY ALL GROUPING SETS ((a, b), (a), ()), CUBE (c), ROLLUP (d)", @@ -1044,7 +1024,7 @@ public void testSelectWithGroupBy() Optional.empty(), ImmutableList.of(), Optional.empty()), - ImmutableList.of(), + ImmutableList.of(), Optional.empty())); assertStatement("SELECT * FROM table1 GROUP BY DISTINCT GROUPING SETS ((a, b), (a), ()), CUBE (c), ROLLUP (d)", @@ -1064,7 +1044,7 @@ public void testSelectWithGroupBy() Optional.empty(), ImmutableList.of(), Optional.empty()), - ImmutableList.of(), + ImmutableList.of(), Optional.empty())); } @@ -1333,14 +1313,14 @@ public void testWith() new WithQuery("b", simpleQuery(selectList(new AllColumns()), table(QualifiedName.of("y"))), Optional.empty())))), new Table(QualifiedName.of("z")), ImmutableList.of(), - Optional.empty())); + Optional.empty())); assertStatement("WITH RECURSIVE a AS (SELECT * FROM x) TABLE y", new Query(Optional.of(new With(true, ImmutableList.of( new WithQuery("a", simpleQuery(selectList(new AllColumns()), table(QualifiedName.of("x"))), Optional.empty())))), new Table(QualifiedName.of("y")), ImmutableList.of(), - Optional.empty())); + Optional.empty())); } @Test @@ -1352,7 +1332,7 @@ public void testImplicitJoin() new Join(Join.Type.IMPLICIT, new Table(QualifiedName.of("a")), new Table(QualifiedName.of("b")), - Optional.empty()))); + Optional.empty()))); } @Test @@ -1515,7 +1495,7 @@ public void testAtTimeZone() Optional.empty(), ImmutableList.of(), Optional.empty()), - ImmutableList.of(), + ImmutableList.of(), Optional.empty())); } @@ -1525,16 +1505,14 @@ public void testLambda() { assertExpression("x -> sin(x)", new LambdaExpression( - ImmutableList.of(new LambdaArgumentDeclaration("x")), - new FunctionCall(QualifiedName.of("sin"), ImmutableList.of(new QualifiedNameReference(QualifiedName.of("x")))) - )); + ImmutableList.of("x"), + new FunctionCall(QualifiedName.of("sin"), ImmutableList.of(new QualifiedNameReference(QualifiedName.of("x")))))); assertExpression("(x, y) -> mod(x, y)", new LambdaExpression( - ImmutableList.of(new LambdaArgumentDeclaration("x"), new LambdaArgumentDeclaration("y")), + ImmutableList.of("x", "y"), new FunctionCall( QualifiedName.of("mod"), - ImmutableList.of(new QualifiedNameReference(QualifiedName.of("x")), new QualifiedNameReference(QualifiedName.of("y")))) - )); + ImmutableList.of(new QualifiedNameReference(QualifiedName.of("x")), new QualifiedNameReference(QualifiedName.of("y")))))); } @Test diff --git a/presto-parser/src/test/java/com/facebook/presto/sql/parser/TestStatementBuilder.java b/presto-parser/src/test/java/com/facebook/presto/sql/parser/TestStatementBuilder.java index 075d76e86ddd..c4e6b2f09a96 100644 --- a/presto-parser/src/test/java/com/facebook/presto/sql/parser/TestStatementBuilder.java +++ b/presto-parser/src/test/java/com/facebook/presto/sql/parser/TestStatementBuilder.java @@ -128,7 +128,7 @@ public void testStatementBuilder() printStatement("select * from a.b.c"); printStatement("select * from a.b.c.e.f.g"); - printStatement("select \"TOTALPRICE\" \"my price\" from \"$MY\"\"ORDERS\""); + printStatement("select \"TOTALPRICE\" \"my price\" from \"ORDERS\""); printStatement("select * from foo tablesample system (10+1)"); printStatement("select * from foo tablesample system (10) join bar tablesample bernoulli (30) on a.id = b.id"); diff --git a/presto-product-tests/README.md b/presto-product-tests/README.md index b12f0570238f..e0091ed73129 100644 --- a/presto-product-tests/README.md +++ b/presto-product-tests/README.md @@ -29,21 +29,13 @@ broken. wget -qO- https://get.docker.com/ | sh ``` -* [`docker-compose >= 1.8.1`](https://docs.docker.com/compose/install/) +* [`docker-compose >= 1.60`](https://docs.docker.com/compose/install/) ``` pip install docker-compose ``` -### OS X using Docker for Mac (macOS 10.10.3 Yosemite or newer) [PREFERRED WAY] - -* Install Docker for Mac: https://docs.docker.com/docker-for-mac/ - -* Add entries in `/etc/hosts` for all services running in docker containers: -`hadoop-master`, `mysql`, `postgres`, `presto-master`. -They should point to your external IP address (shown by `ifconfig` on your Mac (not inside docker)). - -### OS X using Docker Toolbox (macOS 10.8 "Mountain Lion" or newer) [NOT RECOMMENDED] +### OS X (10.8 "Mountain Lion" or newer) * [`VirtualBox >= 5.0`](https://www.virtualbox.org/wiki/Downloads) @@ -64,7 +56,7 @@ Terminal" icon located in ~/Applications/Docker. Note that all commands listed in subsequent parts of this tutorial must be run within such a pre-configured shell. -#### Setting up a Linux VM for Docker Toolbox +#### Setting up a Linux VM for Docker The `docker-toolbox` installation creates a VirtualBox VM called `default`. To run product-tests on the `default` VM, it must be re-configured to use @@ -312,18 +304,13 @@ The format of `/etc/hosts` entries is ` `: Note that above command requires [jq](https://stedolan.github.io/jq/) to be installed in your system - - On OS X: - - Docker for Mac: - Add the following mapping to `/etc/hosts`: ` hadoop-master mysql postgres`. - - - Docker Toolbox: - Add the following mapping to `/etc/hosts`: ` hadoop-master mysql postgres`. - Since Docker containers run inside a Linux VM, on OS X we map the VM IP to - the `hadoop-master`, `mysql` and `postgres` hostnames. To obtain the IP of the Linux VM run: + - On OS X add the following mapping: ` hadoop-master mysql postgres`. + Since Docker containers run inside a Linux VM, on OS X we map the VM IP to + the `hadoop-master`, `mysql` and `postgres` hostnames. To obtain the IP of the Linux VM run: - ``` - docker-machine ip - ``` + ``` + docker-machine ip + ``` 4. [Create a run configuration in IntelliJ](https://www.jetbrains.com/help/idea/2016.1/creating-and-editing-run-debug-configurations.html) with the following parameters: @@ -333,21 +320,12 @@ with the following parameters: - Working directory: `presto-product-tests/conf/presto` - VM options: `-ea -Xmx2G -Dconfig=etc/config.properties -Dlog.levels-file=etc/log.properties -DHADOOP_USER_NAME=hive -Duser.timezone=UTC` -5. MAKE SURE PRESTO CONFIGURATION IS ALIGNED WITH THE ONE IN `presto-product-tests/conf/presto`! +5. Start the Presto server with the newly created run configuration. - If you use custom configuration, make sure to configure the Hive catalog such that it uses a socks proxy running inside the container. - To do so, please make sure your Hive properties file under `etc/catalog/` has the below entry: - ``` - hive.metastore.thrift.client.socks-proxy=hadoop-master:1180 - ``` - Also, `hadoop-master` must point to a correct IP (as described in step 3 of this section). - -6. Start the Presto server with the newly created run configuration. - -7. In IntelliJ, right click on a test method name or test class to run +6. In IntelliJ, right click on a test method name or test class to run or debug the respective test(s). -8. Remember to stop the Hadoop container once debugging is done with the +7. Remember to stop the Hadoop container once debugging is done with the following command: ``` @@ -361,7 +339,7 @@ Some of the product tests are implemented in a manner. Such tests can not be run directly from IntelliJ and the following steps explain how to debug convention based tests: -1. Follow steps [1-6] from the [Debugging Java based tests](#debugging-java-based-tests) +1. Follow steps [1-5] from the [Debugging Java based tests](#debugging-java-based-tests) section. 2. Run a convention based test with the following JVM debug flags: @@ -443,15 +421,3 @@ You can override the default socks proxy port (1180) used by dockerized Hive dep `HIVE_PROXY_PORT` environment variable, e.g. `export HIVE_PROXY_PORT=1180`. This will run all of the dockerized tests using the custom port for the socks proxy. When you change the default socks proxy port (1180) and want to use Hive provided by product tests from outside docker (e.g. access it from Presto running in your IDE), you have to modify the property `hive.metastore.thrift.client.socks-proxy=hadoop-master:1180` in your `hive.properties` file accordingly. Presto inside docker (used while starting tests using `run_on_docker.sh`) will still use default port (1180) though. - -### Malformed reply from SOCKS server - -If you see an error similar to -``` -Failed on local exception: java.net.SocketException: Malformed reply from SOCKS server; Host Details : local host is [...] -``` -Make sure your `/etc/hosts` points to proper IP address (see [Debugging Java based tests](#debugging-java-based-tests), step 3). -Also it's worth confirming that your Hive properties file accounts for the socks proxy used in Hive container (steps 4-5 of [Debugging Java based tests](#debugging-java-based-tests)). - -If `/etc/hosts` entries have changed since the time when Docker containers were provisioned it's worth removing them and re-provisioning. -To do so, use `docker rm` on each container used in product tests. diff --git a/presto-product-tests/bin/run_on_docker.sh b/presto-product-tests/bin/run_on_docker.sh index eb1f867137da..de7aae763450 100755 --- a/presto-product-tests/bin/run_on_docker.sh +++ b/presto-product-tests/bin/run_on_docker.sh @@ -191,12 +191,15 @@ fi # catch terminate signals trap terminate INT TERM EXIT -# start external services -EXTERNAL_SERVICES="hadoop-master mysql postgres cassandra" -environment_compose up -d ${EXTERNAL_SERVICES} +# start hadoop container +environment_compose up -d hadoop-master -# start docker logs for the external services -environment_compose logs --no-color -f ${EXTERNAL_SERVICES} & +# start external database containers +environment_compose up -d mysql +environment_compose up -d postgres + +# start docker logs for hadoop container +environment_compose logs --no-color hadoop-master & HADOOP_LOGS_PID=$! # wait until hadoop processes is started @@ -207,7 +210,7 @@ stop_unnecessary_hadoop_services environment_compose up -d ${PRESTO_SERVICES} # start docker logs for presto containers -environment_compose logs --no-color -f ${PRESTO_SERVICES} & +environment_compose logs --no-color ${PRESTO_SERVICES} & PRESTO_LOGS_PID=$! # wait until presto is started @@ -215,7 +218,7 @@ retry check_presto # run product tests set +e -run_product_tests "$@" +run_product_tests "$*" EXIT_CODE=$? set -e diff --git a/presto-product-tests/conf/docker/common/cassandra.yml b/presto-product-tests/conf/docker/common/cassandra.yml deleted file mode 100644 index 7f48c795f34e..000000000000 --- a/presto-product-tests/conf/docker/common/cassandra.yml +++ /dev/null @@ -1,10 +0,0 @@ -version: '2' -services: - - cassandra: - hostname: cassandra - image: 'cassandra:2.1.15' - ports: - - '9042:9042' - - '9160:9160' - diff --git a/presto-product-tests/conf/docker/multinode/compose.sh b/presto-product-tests/conf/docker/multinode/compose.sh index 53a15a4fc61a..12826a8dee9e 100755 --- a/presto-product-tests/conf/docker/multinode/compose.sh +++ b/presto-product-tests/conf/docker/multinode/compose.sh @@ -5,6 +5,5 @@ source ${BASH_SOURCE%/*}/../common/compose-commons.sh docker-compose \ -f ${BASH_SOURCE%/*}/../common/standard.yml \ -f ${BASH_SOURCE%/*}/../common/jdbc_db.yml \ --f ${BASH_SOURCE%/*}/../common/cassandra.yml \ -f ${BASH_SOURCE%/*}/docker-compose.yml \ "$@" diff --git a/presto-product-tests/conf/docker/singlenode-hdfs-impersonation/compose.sh b/presto-product-tests/conf/docker/singlenode-hdfs-impersonation/compose.sh index 53a15a4fc61a..12826a8dee9e 100755 --- a/presto-product-tests/conf/docker/singlenode-hdfs-impersonation/compose.sh +++ b/presto-product-tests/conf/docker/singlenode-hdfs-impersonation/compose.sh @@ -5,6 +5,5 @@ source ${BASH_SOURCE%/*}/../common/compose-commons.sh docker-compose \ -f ${BASH_SOURCE%/*}/../common/standard.yml \ -f ${BASH_SOURCE%/*}/../common/jdbc_db.yml \ --f ${BASH_SOURCE%/*}/../common/cassandra.yml \ -f ${BASH_SOURCE%/*}/docker-compose.yml \ "$@" diff --git a/presto-product-tests/conf/docker/singlenode-kerberos-hdfs-impersonation/compose.sh b/presto-product-tests/conf/docker/singlenode-kerberos-hdfs-impersonation/compose.sh index 3fc704d18016..65e07cb63852 100755 --- a/presto-product-tests/conf/docker/singlenode-kerberos-hdfs-impersonation/compose.sh +++ b/presto-product-tests/conf/docker/singlenode-kerberos-hdfs-impersonation/compose.sh @@ -6,6 +6,5 @@ docker-compose \ -f ${BASH_SOURCE%/*}/../common/standard.yml \ -f ${BASH_SOURCE%/*}/../common/kerberos.yml \ -f ${BASH_SOURCE%/*}/../common/jdbc_db.yml \ --f ${BASH_SOURCE%/*}/../common/cassandra.yml \ -f ${BASH_SOURCE%/*}/docker-compose.yml \ "$@" diff --git a/presto-product-tests/conf/docker/singlenode-kerberos-hdfs-no-impersonation/compose.sh b/presto-product-tests/conf/docker/singlenode-kerberos-hdfs-no-impersonation/compose.sh index 3fc704d18016..65e07cb63852 100755 --- a/presto-product-tests/conf/docker/singlenode-kerberos-hdfs-no-impersonation/compose.sh +++ b/presto-product-tests/conf/docker/singlenode-kerberos-hdfs-no-impersonation/compose.sh @@ -6,6 +6,5 @@ docker-compose \ -f ${BASH_SOURCE%/*}/../common/standard.yml \ -f ${BASH_SOURCE%/*}/../common/kerberos.yml \ -f ${BASH_SOURCE%/*}/../common/jdbc_db.yml \ --f ${BASH_SOURCE%/*}/../common/cassandra.yml \ -f ${BASH_SOURCE%/*}/docker-compose.yml \ "$@" diff --git a/presto-product-tests/conf/docker/singlenode/compose.sh b/presto-product-tests/conf/docker/singlenode/compose.sh index 53a15a4fc61a..12826a8dee9e 100755 --- a/presto-product-tests/conf/docker/singlenode/compose.sh +++ b/presto-product-tests/conf/docker/singlenode/compose.sh @@ -5,6 +5,5 @@ source ${BASH_SOURCE%/*}/../common/compose-commons.sh docker-compose \ -f ${BASH_SOURCE%/*}/../common/standard.yml \ -f ${BASH_SOURCE%/*}/../common/jdbc_db.yml \ --f ${BASH_SOURCE%/*}/../common/cassandra.yml \ -f ${BASH_SOURCE%/*}/docker-compose.yml \ "$@" diff --git a/presto-product-tests/conf/presto/etc/catalog/cassandra.properties b/presto-product-tests/conf/presto/etc/catalog/cassandra.properties deleted file mode 100644 index 65a89f4785c2..000000000000 --- a/presto-product-tests/conf/presto/etc/catalog/cassandra.properties +++ /dev/null @@ -1,3 +0,0 @@ -connector.name=cassandra -cassandra.contact-points=cassandra -cassandra.schema-cache-ttl=0s diff --git a/presto-product-tests/conf/presto/etc/catalog/hive.properties b/presto-product-tests/conf/presto/etc/catalog/hive.properties index 9806a6c196d6..a490c7fe5897 100644 --- a/presto-product-tests/conf/presto/etc/catalog/hive.properties +++ b/presto-product-tests/conf/presto/etc/catalog/hive.properties @@ -13,4 +13,3 @@ hive.allow-rename-column=true hive.allow-drop-table=true hive.allow-rename-table=true hive.metastore-cache-ttl=0s -hive.fs.cache.max-size=10 diff --git a/presto-product-tests/conf/presto/etc/environment-specific-catalogs/singlenode-hdfs-impersonation/hive.properties b/presto-product-tests/conf/presto/etc/environment-specific-catalogs/singlenode-hdfs-impersonation/hive.properties index da8e0aaa8fc9..7c213d1b0937 100644 --- a/presto-product-tests/conf/presto/etc/environment-specific-catalogs/singlenode-hdfs-impersonation/hive.properties +++ b/presto-product-tests/conf/presto/etc/environment-specific-catalogs/singlenode-hdfs-impersonation/hive.properties @@ -15,4 +15,3 @@ hive.allow-rename-column=true hive.metastore-cache-ttl=0s hive.hdfs.authentication.type=NONE hive.hdfs.impersonation.enabled=true -hive.fs.cache.max-size=10 diff --git a/presto-product-tests/conf/presto/etc/environment-specific-catalogs/singlenode-kerberos-hdfs-impersonation/hive.properties b/presto-product-tests/conf/presto/etc/environment-specific-catalogs/singlenode-kerberos-hdfs-impersonation/hive.properties index 2ef5241f3534..46e0d367b50b 100644 --- a/presto-product-tests/conf/presto/etc/environment-specific-catalogs/singlenode-kerberos-hdfs-impersonation/hive.properties +++ b/presto-product-tests/conf/presto/etc/environment-specific-catalogs/singlenode-kerberos-hdfs-impersonation/hive.properties @@ -19,7 +19,6 @@ hive.hdfs.authentication.type=KERBEROS hive.hdfs.impersonation.enabled=true hive.hdfs.presto.principal=hdfs/hadoop-master@LABS.TERADATA.COM hive.hdfs.presto.keytab=/etc/hadoop/conf/hdfs.keytab -hive.fs.cache.max-size=10 #required for testGrantRevoke() product test hive.security=sql-standard diff --git a/presto-product-tests/conf/presto/etc/environment-specific-catalogs/singlenode-kerberos-hdfs-no-impersonation/hive.properties b/presto-product-tests/conf/presto/etc/environment-specific-catalogs/singlenode-kerberos-hdfs-no-impersonation/hive.properties index 83899454fc80..88059e2254e7 100644 --- a/presto-product-tests/conf/presto/etc/environment-specific-catalogs/singlenode-kerberos-hdfs-no-impersonation/hive.properties +++ b/presto-product-tests/conf/presto/etc/environment-specific-catalogs/singlenode-kerberos-hdfs-no-impersonation/hive.properties @@ -23,4 +23,3 @@ hive.hdfs.authentication.type=KERBEROS hive.hdfs.impersonation.enabled=false hive.hdfs.presto.principal=hdfs/hadoop-master@LABS.TERADATA.COM hive.hdfs.presto.keytab=/etc/hadoop/conf/hdfs.keytab -hive.fs.cache.max-size=10 diff --git a/presto-product-tests/conf/tempto/tempto-configuration-for-docker-default.yaml b/presto-product-tests/conf/tempto/tempto-configuration-for-docker-default.yaml index f2c57de0f3e4..8d54ff12c792 100644 --- a/presto-product-tests/conf/tempto/tempto-configuration-for-docker-default.yaml +++ b/presto-product-tests/conf/tempto/tempto-configuration-for-docker-default.yaml @@ -4,9 +4,3 @@ databases: presto: host: presto-master configured_hdfs_user: hive - cassandra: - host: cassandra - port: 9042 - default_schema: test - skip_create_schema: false - table_manager_type: cassandra diff --git a/presto-product-tests/conf/tempto/tempto-configuration-for-docker-kerberos.yaml b/presto-product-tests/conf/tempto/tempto-configuration-for-docker-kerberos.yaml index 93b68853a79b..d3a4ae2af45c 100644 --- a/presto-product-tests/conf/tempto/tempto-configuration-for-docker-kerberos.yaml +++ b/presto-product-tests/conf/tempto/tempto-configuration-for-docker-kerberos.yaml @@ -16,17 +16,14 @@ databases: presto: host: presto-master - port: 7778 - server_address: https://${databases.presto.host}:${databases.presto.port} - # Use the HTTP interface in JDBC, as Kerberos authentication is not yet supported in there. - jdbc_url: jdbc:presto://${databases.presto.host}:8080/hive/${databases.hive.schema} jdbc_user: hive - https_keystore_path: /etc/presto/conf/keystore.jks - https_keystore_password: password + cli_server_address: https://${databases.presto.host}:7778 cli_authentication: true cli_kerberos_principal: presto-client/presto-master@LABS.TERADATA.COM cli_kerberos_keytab: /etc/presto/conf/presto-client.keytab cli_kerberos_config_path: /etc/krb5.conf cli_kerberos_service_name: presto-server + cli_keystore: /etc/presto/conf/keystore.jks + cli_keystore_password: password cli_kerberos_use_canonical_hostname: false configured_hdfs_user: hdfs diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 0225fb57e04e..6095c86862f3 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -86,10 +86,6 @@ io.airlift http-client - - io.airlift.tpch - tpch - com.facebook.presto.hive hive-apache-jdbc diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/TestGroups.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/TestGroups.java index c512ee0f1b43..da7502b247d5 100644 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/TestGroups.java +++ b/presto-product-tests/src/main/java/com/facebook/presto/tests/TestGroups.java @@ -51,7 +51,6 @@ public final class TestGroups public static final String AUTHORIZATION = "authorization"; public static final String POST_HIVE_1_0_1 = "post_hive_1_0_1"; public static final String HIVE_COERCION = "hive_coercion"; - public static final String CASSANDRA = "cassandra"; private TestGroups() {} } diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/cassandra/CassandraTpchTableDefinitions.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/cassandra/CassandraTpchTableDefinitions.java deleted file mode 100644 index 29b1d15433a1..000000000000 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/cassandra/CassandraTpchTableDefinitions.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.tests.cassandra; - -import com.google.common.collect.ImmutableList; -import com.teradata.tempto.internal.fulfillment.table.cassandra.CassandraTableDefinition; -import com.teradata.tempto.internal.fulfillment.table.cassandra.tpch.CassandraTpchDataSource; -import io.airlift.tpch.TpchTable; - -import java.sql.JDBCType; - -import static com.facebook.presto.tests.cassandra.TestConstants.CONNECTOR_NAME; -import static com.facebook.presto.tests.cassandra.TestConstants.KEY_SPACE; -import static java.sql.JDBCType.BIGINT; -import static java.sql.JDBCType.VARCHAR; - -public class CassandraTpchTableDefinitions -{ - private CassandraTpchTableDefinitions() {} - - public static final ImmutableList NATION_TYPES = ImmutableList.of(BIGINT, VARCHAR, BIGINT, VARCHAR); - - // TpchTable.NATION does provide data in order: nationkey, name, regionkey, comment. Unfortunately Cassandra reorders columns, - // so schema will be: nationkey, comment, name, regionkey (primary key first - nationkey, then alphabetical order: comment, name, regionkey) - // reordering is solved by providing mapping list - public static final CassandraTableDefinition CASSANDRA_NATION = CassandraTableDefinition.cassandraBuilder("nation") - .withDatabase(CONNECTOR_NAME) - .withSchema(KEY_SPACE) - .setCreateTableDDLTemplate("CREATE TABLE %NAME%(" + - " n_nationkey BIGINT," + - " n_name VARCHAR," + - " n_regionkey BIGINT," + - " n_comment VARCHAR," + - " primary key(n_nationkey))") - .setDataSource(new CassandraTpchDataSource(TpchTable.NATION, ImmutableList.of(0, 2, 3, 1), NATION_TYPES, 1.0)) - .build(); -} diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/cassandra/Select.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/cassandra/Select.java deleted file mode 100644 index b5155bdfb933..000000000000 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/cassandra/Select.java +++ /dev/null @@ -1,59 +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.tests.cassandra; - -import com.teradata.tempto.ProductTest; -import com.teradata.tempto.Requirement; -import com.teradata.tempto.RequirementsProvider; -import com.teradata.tempto.configuration.Configuration; -import com.teradata.tempto.query.QueryResult; -import org.testng.annotations.Test; - -import java.sql.SQLException; - -import static com.facebook.presto.tests.TestGroups.CASSANDRA; -import static com.facebook.presto.tests.TpchTableResults.PRESTO_NATION_RESULT; -import static com.facebook.presto.tests.cassandra.CassandraTpchTableDefinitions.CASSANDRA_NATION; -import static com.facebook.presto.tests.cassandra.TestConstants.CONNECTOR_NAME; -import static com.facebook.presto.tests.cassandra.TestConstants.KEY_SPACE; -import static com.facebook.presto.tests.utils.QueryExecutors.onPresto; -import static com.teradata.tempto.assertions.QueryAssert.assertThat; -import static com.teradata.tempto.fulfillment.table.TableRequirements.immutableTable; -import static java.lang.String.format; - -public class Select - extends ProductTest - implements RequirementsProvider -{ - @Override - public Requirement getRequirements(Configuration configuration) - { - return immutableTable(CASSANDRA_NATION); - } - - @Test(groups = CASSANDRA) - public void testSelectNation() - throws SQLException - { - String sql = format( - "SELECT n_nationkey, n_name, n_regionkey, n_comment FROM %s.%s.%s", - CONNECTOR_NAME, - KEY_SPACE, - CASSANDRA_NATION.getName()); - QueryResult queryResult = onPresto() - .executeQuery(sql); - - assertThat(queryResult).matches(PRESTO_NATION_RESULT); - } -} diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/cli/PrestoCliTests.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/cli/PrestoCliTests.java index e5d93ca7f9f6..c01d7d3add63 100644 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/cli/PrestoCliTests.java +++ b/presto-product-tests/src/main/java/com/facebook/presto/tests/cli/PrestoCliTests.java @@ -77,11 +77,11 @@ public class PrestoCliTests private String kerberosServiceName; @Inject(optional = true) - @Named("databases.presto.https_keystore_path") + @Named("databases.presto.cli_keystore") private String keystorePath; @Inject(optional = true) - @Named("databases.presto.https_keystore_password") + @Named("databases.presto.cli_keystore_password") private String keystorePassword; @Inject(optional = true) @@ -167,37 +167,39 @@ public void shouldRunQueryFromFile() private void launchPrestoCliWithServerArgument(String... arguments) throws IOException, InterruptedException { - ImmutableList.Builder prestoClientOptions = ImmutableList.builder(); - prestoClientOptions.add("--server", serverAddress); - prestoClientOptions.add("--user", jdbcUser); - - if (keystorePath != null) { - prestoClientOptions.add("--keystore-path", keystorePath); - } - - if (keystorePassword != null) { - prestoClientOptions.add("--keystore-password", keystorePassword); + if (!authentication) { + ImmutableList.Builder prestoClientOptions = ImmutableList.builder(); + prestoClientOptions.add( + "--server", serverAddress, + "--user", jdbcUser); + prestoClientOptions.add(arguments); + launchPrestoCli(prestoClientOptions.build()); } - - if (authentication) { + else { requireNonNull(kerberosPrincipal, "databases.presto.cli_kerberos_principal is null"); requireNonNull(kerberosKeytab, "databases.presto.cli_kerberos_keytab is null"); requireNonNull(kerberosServiceName, "databases.presto.cli_kerberos_service_name is null"); requireNonNull(kerberosConfigPath, "databases.presto.cli_kerberos_config_path is null"); - - prestoClientOptions.add("--enable-authentication"); - prestoClientOptions.add("--krb5-principal", kerberosPrincipal); - prestoClientOptions.add("--krb5-keytab-path", kerberosKeytab); - prestoClientOptions.add("--krb5-remote-service-name", kerberosServiceName); - prestoClientOptions.add("--krb5-config-path", kerberosConfigPath); - + requireNonNull(keystorePath, "databases.presto.cli_keystore is null"); + requireNonNull(keystorePassword, "databases.presto.cli_keystore_password is null"); + + ImmutableList.Builder prestoClientOptions = ImmutableList.builder(); + prestoClientOptions.add( + "--server", serverAddress, + "--user", jdbcUser, + "--enable-authentication", + "--krb5-principal", kerberosPrincipal, + "--krb5-keytab-path", kerberosKeytab, + "--krb5-remote-service-name", kerberosServiceName, + "--krb5-config-path", kerberosConfigPath, + "--keystore-path", keystorePath, + "--keystore-password", keystorePassword); if (!kerberosUseCanonicalHostname) { prestoClientOptions.add("--krb5-disable-remote-service-hostname-canonicalization"); } + prestoClientOptions.add(arguments); + launchPrestoCli(prestoClientOptions.build()); } - - prestoClientOptions.add(arguments); - launchPrestoCli(prestoClientOptions.build()); } private void launchPrestoCli(String... arguments) diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/AllSimpleTypesTableDefinitions.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/AllSimpleTypesTableDefinitions.java index df2c611657b3..88f3a9f807ea 100644 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/AllSimpleTypesTableDefinitions.java +++ b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/AllSimpleTypesTableDefinitions.java @@ -111,7 +111,7 @@ private static HiveDataSource getTextFileDataSource() public static void populateDataToHiveTable(String tableName) { - onHive().executeQuery(format("INSERT INTO TABLE %s SELECT * FROM %s", + onHive().executeQuery(format("INSERT INTO %s SELECT * FROM %s", tableName, format(tableNameFormat, "textfile"))); } diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/HiveTableDefinitions.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/HiveTableDefinitions.java deleted file mode 100644 index 771e342f088e..000000000000 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/HiveTableDefinitions.java +++ /dev/null @@ -1,66 +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.tests.hive; - -import com.teradata.tempto.fulfillment.table.hive.HiveTableDefinition; - -import static com.teradata.tempto.fulfillment.table.hive.InlineDataSource.createResourceDataSource; - -public class HiveTableDefinitions -{ - private static final String DATA_REVISION = "1"; - - private static final String NATION_PARTITIONED_BY_REGIONKEY_TABLE_NAME = "nation_partitioned_by_regionkey"; - - public static final int NATION_PARTITIONED_BY_REGIONKEY_NUMBER_OF_LINES_PER_SPLIT = 5; - - public static final HiveTableDefinition NATION_PARTITIONED_BY_REGIONKEY = - HiveTableDefinition.builder(NATION_PARTITIONED_BY_REGIONKEY_TABLE_NAME) - .setCreateTableDDLTemplate( - "CREATE %EXTERNAL% TABLE %NAME%(" + - " p_nationkey BIGINT," + - " p_name VARCHAR(25)," + - " p_comment VARCHAR(152)) " + - "PARTITIONED BY (p_regionkey BIGINT)" + - "ROW FORMAT DELIMITED FIELDS TERMINATED BY '|' ") - .addPartition( - "p_regionkey=1", - createResourceDataSource( - NATION_PARTITIONED_BY_REGIONKEY_TABLE_NAME, - DATA_REVISION, - partitionDataFileResource(1))) - .addPartition( - "p_regionkey=2", - createResourceDataSource( - NATION_PARTITIONED_BY_REGIONKEY_TABLE_NAME, - DATA_REVISION, - partitionDataFileResource(2))) - .addPartition( - "p_regionkey=3", - createResourceDataSource( - NATION_PARTITIONED_BY_REGIONKEY_TABLE_NAME, - DATA_REVISION, - partitionDataFileResource(3))) - .build(); - - private static String partitionDataFileResource(int region) - { - return "com/facebook/presto/tests/hive/data/partitioned_nation/nation_region_" + region + ".textfile"; - } - - private HiveTableDefinitions() - { - } -} diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestAllDatatypesFromHiveConnector.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestAllDatatypesFromHiveConnector.java index 32801fc444d3..b8d7ec681835 100644 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestAllDatatypesFromHiveConnector.java +++ b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestAllDatatypesFromHiveConnector.java @@ -36,7 +36,6 @@ import static com.facebook.presto.tests.TestGroups.HIVE_CONNECTOR; import static com.facebook.presto.tests.TestGroups.POST_HIVE_1_0_1; -import static com.facebook.presto.tests.TestGroups.QUARANTINE; import static com.facebook.presto.tests.TestGroups.SMOKE; import static com.facebook.presto.tests.hive.AllSimpleTypesTableDefinitions.ALL_HIVE_SIMPLE_TYPES_ORC; import static com.facebook.presto.tests.hive.AllSimpleTypesTableDefinitions.ALL_HIVE_SIMPLE_TYPES_PARQUET; @@ -120,7 +119,7 @@ public Requirement getRequirements(Configuration configuration) } @Requires(TextRequirements.class) - @Test(groups = {HIVE_CONNECTOR, SMOKE, QUARANTINE}) + @Test(groups = {HIVE_CONNECTOR, SMOKE}) public void testSelectAllDatatypesTextFile() throws SQLException { @@ -152,7 +151,7 @@ public void testSelectAllDatatypesTextFile() } @Requires(OrcRequirements.class) - @Test(groups = {HIVE_CONNECTOR, QUARANTINE}) + @Test(groups = {HIVE_CONNECTOR}) public void testSelectAllDatatypesOrc() throws SQLException { @@ -184,7 +183,7 @@ public void testSelectAllDatatypesOrc() } @Requires(RcfileRequirements.class) - @Test(groups = {HIVE_CONNECTOR, QUARANTINE}) + @Test(groups = {HIVE_CONNECTOR}) public void testSelectAllDatatypesRcfile() throws SQLException { @@ -283,7 +282,7 @@ else if (usingTeradataJdbcDriver(connection)) { } @Requires(ParquetRequirements.class) - @Test(groups = {HIVE_CONNECTOR, POST_HIVE_1_0_1, QUARANTINE}) + @Test(groups = {HIVE_CONNECTOR, POST_HIVE_1_0_1}) public void testSelectAllDatatypesParquetFile() throws SQLException { diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestExternalHiveTable.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestExternalHiveTable.java deleted file mode 100644 index c5cadd1c3b40..000000000000 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestExternalHiveTable.java +++ /dev/null @@ -1,98 +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.tests.hive; - -import com.teradata.tempto.ProductTest; -import com.teradata.tempto.Requirement; -import com.teradata.tempto.RequirementsProvider; -import com.teradata.tempto.configuration.Configuration; -import com.teradata.tempto.fulfillment.table.TableInstance; -import org.testng.annotations.Test; - -import static com.facebook.presto.tests.TestGroups.HIVE_CONNECTOR; -import static com.facebook.presto.tests.hive.HiveTableDefinitions.NATION_PARTITIONED_BY_REGIONKEY; -import static com.facebook.presto.tests.hive.HiveTableDefinitions.NATION_PARTITIONED_BY_REGIONKEY_NUMBER_OF_LINES_PER_SPLIT; -import static com.facebook.presto.tests.utils.QueryExecutors.onHive; -import static com.facebook.presto.tests.utils.QueryExecutors.onPresto; -import static com.teradata.tempto.Requirements.compose; -import static com.teradata.tempto.assertions.QueryAssert.assertThat; -import static com.teradata.tempto.fulfillment.table.MutableTablesState.mutableTablesState; -import static com.teradata.tempto.fulfillment.table.TableRequirements.mutableTable; -import static com.teradata.tempto.fulfillment.table.hive.tpch.TpchTableDefinitions.NATION; - -public class TestExternalHiveTable - extends ProductTest - implements RequirementsProvider -{ - private static final String EXTERNAL_TABLE_NAME = "target_table"; - - public Requirement getRequirements(Configuration configuration) - { - return compose( - mutableTable(NATION), - mutableTable(NATION_PARTITIONED_BY_REGIONKEY)); - } - - @Test(groups = {HIVE_CONNECTOR}) - public void testInsertIntoExternalTable() - { - TableInstance nation = mutableTablesState().get(NATION.getName()); - onHive().executeQuery("DROP TABLE IF EXISTS " + EXTERNAL_TABLE_NAME); - onHive().executeQuery("CREATE EXTERNAL TABLE " + EXTERNAL_TABLE_NAME + " LIKE " + nation.getNameInDatabase()); - assertThat(() -> onPresto().executeQuery( - "INSERT INTO hive.default." + EXTERNAL_TABLE_NAME + " SELECT * FROM hive.default." + nation.getNameInDatabase())) - .failsWithMessage("Cannot write to non-managed Hive table"); - } - - @Test(groups = {HIVE_CONNECTOR}) - public void testDeleteFromExternalTable() - { - TableInstance nation = mutableTablesState().get(NATION.getName()); - onHive().executeQuery("DROP TABLE IF EXISTS " + EXTERNAL_TABLE_NAME); - onHive().executeQuery("CREATE EXTERNAL TABLE " + EXTERNAL_TABLE_NAME + " LIKE " + nation.getNameInDatabase()); - assertThat(() -> onPresto().executeQuery("DELETE FROM hive.default." + EXTERNAL_TABLE_NAME)) - .failsWithMessage("Cannot delete from non-managed Hive table"); - } - - @Test(groups = {HIVE_CONNECTOR}) - public void testDeleteFromExternalPartitionedTableTable() - { - TableInstance nation = mutableTablesState().get(NATION_PARTITIONED_BY_REGIONKEY.getName()); - onHive().executeQuery("DROP TABLE IF EXISTS " + EXTERNAL_TABLE_NAME); - onHive().executeQuery("CREATE EXTERNAL TABLE " + EXTERNAL_TABLE_NAME + " LIKE " + nation.getNameInDatabase() + " LOCATION '/tmp/" + EXTERNAL_TABLE_NAME + "_" + nation.getNameInDatabase() + "'"); - insertNationPartition(nation, 1); - insertNationPartition(nation, 2); - insertNationPartition(nation, 3); - assertThat(onPresto().executeQuery("SELECT * FROM " + EXTERNAL_TABLE_NAME)) - .hasRowsCount(3 * NATION_PARTITIONED_BY_REGIONKEY_NUMBER_OF_LINES_PER_SPLIT); - - assertThat(() -> onPresto().executeQuery("DELETE FROM hive.default." + EXTERNAL_TABLE_NAME + " WHERE p_name IS NOT NULL")) - .failsWithMessage("This connector only supports delete where one or more partitions are deleted entirely"); - - onPresto().executeQuery("DELETE FROM hive.default." + EXTERNAL_TABLE_NAME + " WHERE p_regionkey = 1"); - assertThat(onPresto().executeQuery("SELECT * FROM " + EXTERNAL_TABLE_NAME)) - .hasRowsCount(2 * NATION_PARTITIONED_BY_REGIONKEY_NUMBER_OF_LINES_PER_SPLIT); - - onPresto().executeQuery("DELETE FROM hive.default." + EXTERNAL_TABLE_NAME); - assertThat(onPresto().executeQuery("SELECT * FROM " + EXTERNAL_TABLE_NAME)).hasRowsCount(0); - } - - private void insertNationPartition(TableInstance nation, int partition) - { - onHive().executeQuery( - "INSERT INTO TABLE " + EXTERNAL_TABLE_NAME + " PARTITION (p_regionkey=" + partition + ")" - + " SELECT p_nationkey, p_name, p_comment FROM " + nation.getNameInDatabase() - + " WHERE p_regionkey=" + partition); - } -} diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveCoercion.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveCoercion.java index a117a2b377b6..4f1c57365197 100644 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveCoercion.java +++ b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveCoercion.java @@ -214,7 +214,7 @@ private void doTestHiveCoercion(HiveTableDefinition tableDefinition) { String tableName = mutableTableInstanceOf(tableDefinition).getNameInDatabase(); - executeHiveQuery(format("INSERT INTO TABLE %s " + + executeHiveQuery(format("INSERT INTO %s " + "PARTITION (id=1) " + "VALUES" + "(-1, 2, -3, 100, -101, 2323, 12345, '-1025', 0.5)," + diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestTablePartitioningInsertInto.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestTablePartitioningInsertInto.java index ea79ef60afcb..0850be6d3187 100644 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestTablePartitioningInsertInto.java +++ b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestTablePartitioningInsertInto.java @@ -16,18 +16,16 @@ import com.teradata.tempto.Requirement; import com.teradata.tempto.RequirementsProvider; import com.teradata.tempto.configuration.Configuration; +import com.teradata.tempto.fulfillment.table.MutableTableRequirement; +import com.teradata.tempto.fulfillment.table.hive.HiveTableDefinition; import com.teradata.tempto.query.QueryResult; import org.testng.annotations.Test; import static com.facebook.presto.tests.TestGroups.HIVE_CONNECTOR; import static com.facebook.presto.tests.TestGroups.SMOKE; -import static com.facebook.presto.tests.hive.HiveTableDefinitions.NATION_PARTITIONED_BY_REGIONKEY; -import static com.facebook.presto.tests.hive.HiveTableDefinitions.NATION_PARTITIONED_BY_REGIONKEY_NUMBER_OF_LINES_PER_SPLIT; import static com.teradata.tempto.Requirements.compose; -import static com.teradata.tempto.fulfillment.table.MutableTableRequirement.State.CREATED; import static com.teradata.tempto.fulfillment.table.MutableTablesState.mutableTablesState; -import static com.teradata.tempto.fulfillment.table.TableRequirements.mutableTable; -import static com.teradata.tempto.fulfillment.table.hive.tpch.TpchTableDefinitions.NATION; +import static com.teradata.tempto.fulfillment.table.hive.InlineDataSource.createResourceDataSource; import static com.teradata.tempto.query.QueryExecutor.query; import static com.teradata.tempto.query.QueryType.UPDATE; import static org.assertj.core.api.Assertions.assertThat; @@ -36,14 +34,48 @@ public class TestTablePartitioningInsertInto extends HivePartitioningTest implements RequirementsProvider { + private static final String PARTITIONED_NATION_NAME = "partitioned_nation_read_test"; private static final String TARGET_NATION_NAME = "target_nation_test"; + private static final int NUMBER_OF_LINES_PER_SPLIT = 5; + private static final String DATA_REVISION = "1"; + private static final HiveTableDefinition PARTITIONED_NATION = + HiveTableDefinition.builder(PARTITIONED_NATION_NAME) + .setCreateTableDDLTemplate("" + + "CREATE %EXTERNAL% TABLE %NAME%(" + + " p_nationkey BIGINT," + + " p_name STRING," + + " p_comment STRING) " + + "PARTITIONED BY (p_regionkey INT)" + + "ROW FORMAT DELIMITED FIELDS TERMINATED BY '|' ") + .addPartition("p_regionkey=1", createResourceDataSource(PARTITIONED_NATION_NAME, DATA_REVISION, partitionDataFileResource(1))) + .addPartition("p_regionkey=2", createResourceDataSource(PARTITIONED_NATION_NAME, DATA_REVISION, partitionDataFileResource(2))) + .addPartition("p_regionkey=3", createResourceDataSource(PARTITIONED_NATION_NAME, DATA_REVISION, partitionDataFileResource(3))) + .build(); + + private static final HiveTableDefinition TARGET_NATION = + HiveTableDefinition.builder(TARGET_NATION_NAME) + .setCreateTableDDLTemplate("" + + "CREATE %EXTERNAL% TABLE %NAME%(" + + " p_nationkey BIGINT," + + " p_name STRING," + + " p_comment STRING," + + " p_regionkey INT) " + + "ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'") + .setNoData() + .build(); + + private static String partitionDataFileResource(int region) + { + return "com/facebook/presto/tests/hive/data/partitioned_nation/nation_region_" + region + ".textfile"; + } + @Override public Requirement getRequirements(Configuration configuration) { return compose( - mutableTable(NATION_PARTITIONED_BY_REGIONKEY), - mutableTable(NATION, TARGET_NATION_NAME, CREATED)); + MutableTableRequirement.builder(PARTITIONED_NATION).build(), + MutableTableRequirement.builder(TARGET_NATION).build()); } @Test(groups = {HIVE_CONNECTOR, SMOKE}) @@ -51,37 +83,32 @@ public void selectFromPartitionedNation() throws Exception { // read all data - testQuerySplitsNumber("p_nationkey < 40", 3); + testQuerySplitsNumber("INSERT INTO %s SELECT * FROM %s WHERE p_nationkey < 40", NUMBER_OF_LINES_PER_SPLIT * 3); // read no partitions - testQuerySplitsNumber("p_regionkey = 42", 0); + testQuerySplitsNumber("INSERT INTO %s SELECT * FROM %s WHERE p_regionkey = 42", 0); // read one partition - testQuerySplitsNumber("p_regionkey = 2 AND p_nationkey < 40", 1); + testQuerySplitsNumber("INSERT INTO %s SELECT * FROM %s WHERE p_regionkey = 2 AND p_nationkey < 40", NUMBER_OF_LINES_PER_SPLIT * 1); // read two partitions - testQuerySplitsNumber("p_regionkey = 2 AND p_nationkey < 40 or p_regionkey = 3", 2); + testQuerySplitsNumber("INSERT INTO %s SELECT * FROM %s WHERE p_regionkey = 2 AND p_nationkey < 40 or p_regionkey = 3", NUMBER_OF_LINES_PER_SPLIT * 2); // read all (three) partitions - testQuerySplitsNumber("p_regionkey = 2 OR p_nationkey < 40", 3); + testQuerySplitsNumber("INSERT INTO %s SELECT * FROM %s WHERE p_regionkey = 2 OR p_nationkey < 40", NUMBER_OF_LINES_PER_SPLIT * 3); // range read two partitions - testQuerySplitsNumber("p_regionkey <= 2", 2); - testQuerySplitsNumber("p_regionkey <= 1 OR p_regionkey >= 3", 2); + testQuerySplitsNumber("INSERT INTO %s SELECT * FROM %s WHERE p_regionkey <= 2", NUMBER_OF_LINES_PER_SPLIT * 2); + testQuerySplitsNumber("INSERT INTO %s SELECT * FROM %s WHERE p_regionkey <= 1 OR p_regionkey >= 3", NUMBER_OF_LINES_PER_SPLIT * 2); } - private void testQuerySplitsNumber(String condition, int expectedProcessedSplits) + private void testQuerySplitsNumber(String query, int expectedProcessedLines) throws Exception { - String partitionedNation = mutableTablesState().get(NATION_PARTITIONED_BY_REGIONKEY.getTableHandle()).getNameInDatabase(); + String partitionedNation = mutableTablesState().get(PARTITIONED_NATION_NAME).getNameInDatabase(); String targetNation = mutableTablesState().get(TARGET_NATION_NAME).getNameInDatabase(); - String query = String.format( - "INSERT INTO %s SELECT p_nationkey, p_name, p_regionkey, p_comment FROM %s WHERE %s", - targetNation, - partitionedNation, - condition); - QueryResult queryResult = query(query, UPDATE); + String sqlStatement = String.format(query, targetNation, partitionedNation); + QueryResult queryResult = query(sqlStatement, UPDATE); - long processedLinesCount = getProcessedLinesCount(query, queryResult); - int expectedLinesCount = expectedProcessedSplits * NATION_PARTITIONED_BY_REGIONKEY_NUMBER_OF_LINES_PER_SPLIT; - assertThat(processedLinesCount).isEqualTo(expectedLinesCount); + long processedLinesCount = getProcessedLinesCount(sqlStatement, queryResult); + assertThat(processedLinesCount).isEqualTo(expectedProcessedLines); } } diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/querystats/QueryStatsClientModuleProvider.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/querystats/QueryStatsClientModuleProvider.java index f6a8443303d5..c7aea3cc1a94 100644 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/querystats/QueryStatsClientModuleProvider.java +++ b/presto-product-tests/src/main/java/com/facebook/presto/tests/querystats/QueryStatsClientModuleProvider.java @@ -23,7 +23,6 @@ import com.teradata.tempto.configuration.Configuration; import com.teradata.tempto.initialization.AutoModuleProvider; import com.teradata.tempto.initialization.SuiteModuleProvider; -import io.airlift.http.client.HttpClientConfig; import io.airlift.http.client.jetty.JettyHttpClient; import io.airlift.json.ObjectMapperProvider; @@ -51,14 +50,11 @@ protected void configure() @Inject @Provides @Exposed - QueryStatsClient getQueryStatsClient(ObjectMapper objectMapper, @Named("databases.presto.server_address") String serverAddress) + QueryStatsClient getQueryStatsClient(ObjectMapper objectMapper, @Named("presto_rest.base_uri") String prestoRestInterfaceBaseUri) { // @Singleton does not work due: https://github.com/prestodb/tempto/issues/94 if (httpQueryStatsClient == null) { - HttpClientConfig httpClientConfig = new HttpClientConfig(); - httpClientConfig.setKeyStorePath(configuration.getString("databases.presto.https_keystore_path").orElse(null)); - httpClientConfig.setKeyStorePassword(configuration.getString("databases.presto.https_keystore_password").orElse(null)); - httpQueryStatsClient = new HttpQueryStatsClient(new JettyHttpClient(httpClientConfig), objectMapper, URI.create(serverAddress)); + httpQueryStatsClient = new HttpQueryStatsClient(new JettyHttpClient(), objectMapper, URI.create(prestoRestInterfaceBaseUri)); } return httpQueryStatsClient; } diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/utils/QueryExecutors.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/utils/QueryExecutors.java index 8d00f2f33e65..e5a959a1fea8 100644 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/utils/QueryExecutors.java +++ b/presto-product-tests/src/main/java/com/facebook/presto/tests/utils/QueryExecutors.java @@ -29,10 +29,5 @@ public static QueryExecutor connectToPresto(String prestoConfig) return testContext().getDependency(QueryExecutor.class, prestoConfig); } - public static QueryExecutor onHive() - { - return testContext().getDependency(QueryExecutor.class, "hive"); - } - private QueryExecutors() {} } diff --git a/presto-product-tests/src/main/resources/sql-tests/testcases/catalog/describe.result b/presto-product-tests/src/main/resources/sql-tests/testcases/catalog/describe.result index ba8b0dc91ae5..268754cdffa1 100644 --- a/presto-product-tests/src/main/resources/sql-tests/testcases/catalog/describe.result +++ b/presto-product-tests/src/main/resources/sql-tests/testcases/catalog/describe.result @@ -1,5 +1,5 @@ -- delimiter: |; trimValues:true; -n_nationkey | bigint | | | -n_name | varchar(25) | | | -n_regionkey | bigint | | | -n_comment | varchar(152) | | | +n_nationkey | bigint | | +n_name | varchar(25) | | +n_regionkey | bigint | | +n_comment | varchar(152) | | diff --git a/presto-product-tests/src/main/resources/sql-tests/testcases/catalog/showColumns.result b/presto-product-tests/src/main/resources/sql-tests/testcases/catalog/showColumns.result index 2dadcf0c8316..238f852de6f6 100644 --- a/presto-product-tests/src/main/resources/sql-tests/testcases/catalog/showColumns.result +++ b/presto-product-tests/src/main/resources/sql-tests/testcases/catalog/showColumns.result @@ -1,5 +1,5 @@ --- delimiter: |; ignoreOrder: true; ignoreExcessRows:true; trimValues:true; - node_id | varchar | | | - http_uri | varchar | | | - node_version | varchar | | | - state | varchar | | | +-- delimiter: |; ignoreOrder: true; ignoreExcessRows:true; trimValues:true; + node_id | varchar | | + http_uri | varchar | | + node_version | varchar | | + state | varchar | | diff --git a/presto-product-tests/src/main/resources/sql-tests/testcases/connectors/mysql/describe_real_table.sql b/presto-product-tests/src/main/resources/sql-tests/testcases/connectors/mysql/describe_real_table.sql index 6ce2acc26025..a1b17ac70e1e 100644 --- a/presto-product-tests/src/main/resources/sql-tests/testcases/connectors/mysql/describe_real_table.sql +++ b/presto-product-tests/src/main/resources/sql-tests/testcases/connectors/mysql/describe_real_table.sql @@ -3,8 +3,8 @@ describe mysql.test.real_table_mysql --! -- delimiter: |; trimValues: true; ignoreOrder: true; -id_employee | integer | | | -salary | double | | | -bonus | real | | | -tip | real | | | -tip2 | double | | | +id_employee | integer | | +salary | double | | +bonus | real | | +tip | real | | +tip2 | double | | diff --git a/presto-product-tests/src/main/resources/sql-tests/testcases/connectors/mysql/describe_table.sql b/presto-product-tests/src/main/resources/sql-tests/testcases/connectors/mysql/describe_table.sql index 9b19da1718e5..491af5f7b8d9 100644 --- a/presto-product-tests/src/main/resources/sql-tests/testcases/connectors/mysql/describe_table.sql +++ b/presto-product-tests/src/main/resources/sql-tests/testcases/connectors/mysql/describe_table.sql @@ -3,11 +3,11 @@ describe mysql.test.workers_mysql --! -- delimiter: |; trimValues: true; ignoreOrder: true; -id_employee | integer | | | -first_name | varchar(32) | | | -last_name | varchar(32) | | | -date_of_employment | date | | | -department | tinyint | | | -id_department | integer | | | -name | varchar(32) | | | -salary | integer | | | +id_employee | integer | | +first_name | varchar(32) | | +last_name | varchar(32) | | +date_of_employment | date | | +department | tinyint | | +id_department | integer | | +name | varchar(32) | | +salary | integer | | diff --git a/presto-product-tests/src/main/resources/sql-tests/testcases/connectors/postgresql/describe_real_table.sql b/presto-product-tests/src/main/resources/sql-tests/testcases/connectors/postgresql/describe_real_table.sql index cac4f5ceb658..9b656ce15e70 100644 --- a/presto-product-tests/src/main/resources/sql-tests/testcases/connectors/postgresql/describe_real_table.sql +++ b/presto-product-tests/src/main/resources/sql-tests/testcases/connectors/postgresql/describe_real_table.sql @@ -3,8 +3,8 @@ describe postgresql.public.real_table_psql --! -- delimiter: |; trimValues: true; ignoreOrder: true; -id_employee | integer | | | -salary | real | | | -bonus | double | | | -tip | real | | | -tip2 | double | | | +id_employee | integer | | +salary | real | | +bonus | double | | +tip | real | | +tip2 | double | | diff --git a/presto-product-tests/src/main/resources/sql-tests/testcases/connectors/postgresql/describe_table.sql b/presto-product-tests/src/main/resources/sql-tests/testcases/connectors/postgresql/describe_table.sql index fed7befef145..930eb82d5aee 100644 --- a/presto-product-tests/src/main/resources/sql-tests/testcases/connectors/postgresql/describe_table.sql +++ b/presto-product-tests/src/main/resources/sql-tests/testcases/connectors/postgresql/describe_table.sql @@ -3,11 +3,11 @@ describe postgresql.public.workers_psql --! -- delimiter: |; trimValues: true; ignoreOrder: true; -id_employee | integer | | | -first_name | varchar(32) | | | -last_name | varchar(32) | | | -date_of_employment | date | | | -department | integer | | | -id_department | integer | | | -name | varchar(32) | | | -salary | integer | | | +id_employee | integer | | +first_name | varchar(32) | | +last_name | varchar(32) | | +date_of_employment | date | | +department | integer | | +id_department | integer | | +name | varchar(32) | | +salary | integer | | diff --git a/presto-product-tests/src/main/resources/sql-tests/testcases/hive_tpch/q15.result b/presto-product-tests/src/main/resources/sql-tests/testcases/hive_tpch/q15.result index 0af7d95b272f..64818bcc81db 100644 --- a/presto-product-tests/src/main/resources/sql-tests/testcases/hive_tpch/q15.result +++ b/presto-product-tests/src/main/resources/sql-tests/testcases/hive_tpch/q15.result @@ -1,2 +1,2 @@ --- delimiter: |; ignoreOrder: false; types: BIGINT|LONGNVARCHAR|LONGNVARCHAR|LONGNVARCHAR|DECIMAL +-- delimiter: |; ignoreOrder: false; types: BIGINT|LONGNVARCHAR|LONGNVARCHAR|LONGNVARCHAR|DOUBLE 8449|Supplier#000008449|Wp34zim9qYFbVctdW|20-469-856-8873|1772627.2087| diff --git a/presto-product-tests/src/main/resources/sql-tests/testcases/hive_tpch/q15.sql b/presto-product-tests/src/main/resources/sql-tests/testcases/hive_tpch/q15.sql index bdb9e4389882..e69c00022339 100644 --- a/presto-product-tests/src/main/resources/sql-tests/testcases/hive_tpch/q15.sql +++ b/presto-product-tests/src/main/resources/sql-tests/testcases/hive_tpch/q15.sql @@ -1,4 +1,4 @@ --- database: presto; groups: tpch; tables: lineitem,supplier +-- database: presto; groups: tpch,quarantine; tables: lineitem,supplier CREATE OR REPLACE VIEW revenue AS SELECT l_suppkey AS supplier_no, diff --git a/presto-product-tests/src/main/resources/sql-tests/testcases/hive_tpch/q6.result b/presto-product-tests/src/main/resources/sql-tests/testcases/hive_tpch/q6.result index 8cb81f637d73..123de901556e 100644 --- a/presto-product-tests/src/main/resources/sql-tests/testcases/hive_tpch/q6.result +++ b/presto-product-tests/src/main/resources/sql-tests/testcases/hive_tpch/q6.result @@ -1,2 +1,2 @@ --- delimiter: |; ignoreOrder: false; types: DECIMAL +-- delimiter: |; ignoreOrder: false; types: DOUBLE 123141078.2283| diff --git a/presto-product-tests/src/main/resources/sql-tests/testcases/hive_tpch/q6.sql b/presto-product-tests/src/main/resources/sql-tests/testcases/hive_tpch/q6.sql index d58319e4d2d4..e81f8e8a5b54 100644 --- a/presto-product-tests/src/main/resources/sql-tests/testcases/hive_tpch/q6.sql +++ b/presto-product-tests/src/main/resources/sql-tests/testcases/hive_tpch/q6.sql @@ -1,9 +1,9 @@ --- database: presto; groups: tpch; tables: lineitem +-- database: presto; groups: tpch,quarantine; tables: lineitem SELECT sum(l_extendedprice * l_discount) AS revenue FROM lineitem WHERE l_shipdate >= DATE '1994-01-01' AND l_shipdate < DATE '1994-01-01' + INTERVAL '1' YEAR -AND l_discount BETWEEN decimal '0.06' - decimal '0.01' AND decimal '0.06' + decimal '0.01' +AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01 AND l_quantity < 24 diff --git a/presto-product-tests/src/main/resources/tempto-configuration.yaml b/presto-product-tests/src/main/resources/tempto-configuration.yaml index d9b635afb797..d074d3eb8e74 100644 --- a/presto-product-tests/src/main/resources/tempto-configuration.yaml +++ b/presto-product-tests/src/main/resources/tempto-configuration.yaml @@ -4,6 +4,9 @@ hdfs: host: ${databases.hive.host} port: 50070 +presto_rest: + base_uri: http://${databases.presto.server_address}/ + databases: default: alias: presto @@ -23,37 +26,30 @@ databases: presto: host: localhost - port: 8080 - server_address: http://${databases.presto.host}:${databases.presto.port} + server_address: ${databases.presto.host}:8080 jdbc_driver_class: com.facebook.presto.jdbc.PrestoDriver - jdbc_url: jdbc:presto://${databases.presto.host}:${databases.presto.port}/hive/${databases.hive.schema} + jdbc_url: jdbc:presto://${databases.presto.server_address}/hive/${databases.hive.schema} jdbc_user: hdfs jdbc_password: na jdbc_pooling: false alice@presto: - host: ${databases.presto.host} - port: ${databases.presto.port} - server_address: ${databases.presto.server_address} + host: localhost + server_address: ${databases.presto.host}:8080 jdbc_driver_class: ${databases.presto.jdbc_driver_class} - jdbc_url: ${databases.presto.jdbc_url} + jdbc_url: jdbc:presto://${databases.presto.server_address}/hive/${databases.hive.schema} jdbc_user: alice jdbc_password: na jdbc_pooling: false - https_keystore_path: ${databases.presto.https_keystore_path} - https_keystore_password: ${databases.presto.https_keystore_password} bob@presto: - host: ${databases.presto.host} - port: ${databases.presto.port} - server_address: ${databases.presto.server_address} + host: localhost + server_address: ${databases.presto.host}:8080 jdbc_driver_class: ${databases.presto.jdbc_driver_class} - jdbc_url: ${databases.presto.jdbc_url} + jdbc_url: jdbc:presto://${databases.presto.server_address}/hive/${databases.hive.schema} jdbc_user: bob jdbc_password: na jdbc_pooling: false - https_keystore_path: ${databases.presto.https_keystore_path} - https_keystore_password: ${databases.presto.https_keystore_password} mysql: jdbc_driver_class: com.mysql.jdbc.Driver diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorBucketFunction.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorBucketFunction.java index d8a8aaf97098..72fa7ba0e505 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorBucketFunction.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorBucketFunction.java @@ -15,32 +15,21 @@ import com.facebook.presto.spi.BucketFunction; import com.facebook.presto.spi.Page; -import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.block.Block; -import com.facebook.presto.spi.type.Type; import io.airlift.slice.XxHash64; -import java.util.List; - -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.IntegerType.INTEGER; -import static com.facebook.presto.spi.type.Varchars.isVarcharType; import static com.google.common.base.Preconditions.checkArgument; public class RaptorBucketFunction implements BucketFunction { - private final HashFunction[] functions; private final int bucketCount; - public RaptorBucketFunction(int bucketCount, List types) + public RaptorBucketFunction(int bucketCount) { checkArgument(bucketCount > 0, "bucketCount must be at least one"); this.bucketCount = bucketCount; - this.functions = types.stream() - .map(RaptorBucketFunction::getHashFunction) - .toArray(HashFunction[]::new); } @SuppressWarnings("NumericCastThatLosesPrecision") @@ -50,49 +39,10 @@ public int getBucket(Page page, int position) long hash = 0; for (int i = 0; i < page.getChannelCount(); i++) { Block block = page.getBlock(i); - long value = functions[i].hash(block, position); - hash = (hash * 31) + value; + long value = BIGINT.getLong(block, position); + hash = (hash * 31) + XxHash64.hash(value); } int value = (int) (hash & Integer.MAX_VALUE); return value % bucketCount; } - - public static void validateBucketType(Type type) - { - getHashFunction(type); - } - - private static HashFunction getHashFunction(Type type) - { - if (type.equals(BIGINT)) { - return bigintHashFunction(); - } - if (type.equals(INTEGER)) { - return intHashFunction(); - } - if (isVarcharType(type)) { - return varcharHashFunction(); - } - throw new PrestoException(NOT_SUPPORTED, "Bucketing is supported for bigint, integer and varchar, not " + type.getDisplayName()); - } - - private static HashFunction bigintHashFunction() - { - return (block, position) -> XxHash64.hash(BIGINT.getLong(block, position)); - } - - private static HashFunction intHashFunction() - { - return (block, position) -> XxHash64.hash(INTEGER.getLong(block, position)); - } - - private static HashFunction varcharHashFunction() - { - return (block, position) -> XxHash64.hash(block.getSlice(position, 0, block.getLength(position))); - } - - private interface HashFunction - { - long hash(Block block, int position); - } } diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorMetadata.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorMetadata.java index b9132dda5c5d..12c4472e6371 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorMetadata.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorMetadata.java @@ -72,7 +72,6 @@ import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; -import static com.facebook.presto.raptor.RaptorBucketFunction.validateBucketType; import static com.facebook.presto.raptor.RaptorColumnHandle.BUCKET_NUMBER_COLUMN_NAME; import static com.facebook.presto.raptor.RaptorColumnHandle.SHARD_UUID_COLUMN_NAME; import static com.facebook.presto.raptor.RaptorColumnHandle.SHARD_UUID_COLUMN_TYPE; @@ -105,6 +104,7 @@ import static com.facebook.presto.spi.StandardErrorCode.NOT_FOUND; import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; import static com.facebook.presto.spi.block.SortOrder.ASC_NULLS_FIRST; +import static com.facebook.presto.spi.type.BigintType.BIGINT; import static com.facebook.presto.spi.type.DateType.DATE; import static com.facebook.presto.spi.type.IntegerType.INTEGER; import static com.facebook.presto.spi.type.TimestampType.TIMESTAMP; @@ -297,7 +297,6 @@ private ConnectorTableLayout getTableLayout(ConnectorSession session, RaptorTabl } List bucketColumnHandles = getBucketColumnHandles(handle.getTableId()); - RaptorPartitioningHandle partitioning = getPartitioningHandle(handle.getDistributionId().getAsLong()); boolean oneSplitPerBucket = handle.getBucketCount().getAsInt() >= getOneSplitPerBucketThreshold(session); @@ -355,7 +354,9 @@ private Optional getOrCreateDistribution(Map bucketColumnTypes = ImmutableList.builder(); for (RaptorColumnHandle column : bucketColumnHandles) { - validateBucketType(column.getColumnType()); + if (!column.getColumnType().equals(BIGINT)) { + throw new PrestoException(NOT_SUPPORTED, "Bucketing is only supported for BIGINT columns"); + } bucketColumnTypes.add(column.getColumnType()); } diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorNodePartitioningProvider.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorNodePartitioningProvider.java index f707f0f46f81..0ed16593c5c7 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorNodePartitioningProvider.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorNodePartitioningProvider.java @@ -74,6 +74,6 @@ public ToIntFunction getSplitBucketFunction(ConnectorTransaction @Override public BucketFunction getBucketFunction(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorPartitioningHandle partitioning, List partitionChannelTypes, int bucketCount) { - return new RaptorBucketFunction(bucketCount, partitionChannelTypes); + return new RaptorBucketFunction(bucketCount); } } diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorPageSink.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorPageSink.java index af3f0cf8a89d..b07a7a00d009 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorPageSink.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/RaptorPageSink.java @@ -21,6 +21,7 @@ import com.facebook.presto.spi.Page; import com.facebook.presto.spi.PageBuilder; import com.facebook.presto.spi.PageSorter; +import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.block.Block; import com.facebook.presto.spi.block.BlockBuilder; import com.facebook.presto.spi.block.SortOrder; @@ -34,13 +35,14 @@ import it.unimi.dsi.fastutil.longs.Long2ObjectMap; import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap; -import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.Optional; import java.util.OptionalInt; import java.util.concurrent.CompletableFuture; +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.DateType.DATE; import static com.facebook.presto.spi.type.TimestampType.TIMESTAMP; import static com.google.common.base.Preconditions.checkArgument; @@ -96,6 +98,12 @@ public RaptorPageSink( this.bucketCount = bucketCount; this.bucketFields = bucketColumnIds.stream().mapToInt(columnIds::indexOf).toArray(); + for (int field : bucketFields) { + if (!columnTypes.get(field).equals(BIGINT)) { + throw new PrestoException(NOT_SUPPORTED, "Bucketing is only supported for BIGINT columns"); + } + } + if (temporalColumnHandle.isPresent() && columnIds.contains(temporalColumnHandle.get().getColumnId())) { temporalColumnIndex = OptionalInt.of(columnIds.indexOf(temporalColumnHandle.get().getColumnId())); temporalColumnType = Optional.of(columnTypes.get(temporalColumnIndex.getAsInt())); @@ -205,12 +213,8 @@ public PartitionedPageWriter() checkArgument(temporalColumnIndex.isPresent() == temporalColumnType.isPresent(), "temporalColumnIndex and temporalColumnType must be both present or absent"); - List bucketTypes = Arrays.stream(bucketFields) - .mapToObj(columnTypes::get) - .collect(toList()); - - this.bucketFunction = bucketCount.isPresent() ? Optional.of(new RaptorBucketFunction(bucketCount.getAsInt(), bucketTypes)) : Optional.empty(); - this.temporalFunction = temporalColumnType.map(TemporalFunction::create); + this.bucketFunction = bucketCount.isPresent() ? Optional.of(new RaptorBucketFunction(bucketCount.getAsInt())) : Optional.empty(); + this.temporalFunction = temporalColumnType.map(type -> TemporalFunction.create(temporalColumnType.get())); } @Override diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/DatabaseShardManager.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/DatabaseShardManager.java index 7745c424ba7e..e09f065c404f 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/DatabaseShardManager.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/DatabaseShardManager.java @@ -72,7 +72,6 @@ import static com.facebook.presto.raptor.util.ArrayUtil.intArrayFromBytes; import static com.facebook.presto.raptor.util.ArrayUtil.intArrayToBytes; import static com.facebook.presto.raptor.util.DatabaseUtil.bindOptionalInt; -import static com.facebook.presto.raptor.util.DatabaseUtil.isSyntaxOrAccessError; import static com.facebook.presto.raptor.util.DatabaseUtil.metadataError; import static com.facebook.presto.raptor.util.DatabaseUtil.runIgnoringConstraintViolation; import static com.facebook.presto.raptor.util.DatabaseUtil.runTransaction; @@ -103,7 +102,6 @@ public class DatabaseShardManager private static final Logger log = Logger.get(DatabaseShardManager.class); private static final String INDEX_TABLE_PREFIX = "x_shards_t"; - private static final int MAX_ADD_COLUMN_ATTEMPTS = 100; private final IDBI dbi; private final DaoSupplier shardDaoSupplier; @@ -275,21 +273,11 @@ public void addColumn(long tableId, ColumnInfo column) minColumn(column.getColumnId()), columnType, maxColumn(column.getColumnId()), columnType); - int attempts = 0; - while (true) { - attempts++; - try (Handle handle = dbi.open()) { - handle.execute(sql); - } - catch (DBIException e) { - if (isSyntaxOrAccessError(e)) { - // exit when column already exists - return; - } - if (attempts >= MAX_ADD_COLUMN_ATTEMPTS) { - throw metadataError(e); - } - } + try (Handle handle = dbi.open()) { + handle.execute(sql); + } + catch (DBIException e) { + throw metadataError(e); } } diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/SchemaDao.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/SchemaDao.java index f033eb0bbf3e..fec7dbb41412 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/SchemaDao.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/metadata/SchemaDao.java @@ -116,8 +116,7 @@ public interface SchemaDao " successful BOOLEAN,\n" + " start_time DATETIME NOT NULL,\n" + " end_time DATETIME,\n" + - " UNIQUE (successful, start_time, transaction_id),\n" + - " UNIQUE (end_time, transaction_id, successful)\n" + + " UNIQUE (successful, start_time, transaction_id)\n" + ")") void createTableTransactions(); diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/ShardRecoveryManager.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/ShardRecoveryManager.java index 360fa1487cf1..f504cdddc99c 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/ShardRecoveryManager.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/storage/ShardRecoveryManager.java @@ -142,9 +142,7 @@ private void enqueueMissingShards() { missingShardExecutor.scheduleWithFixedDelay(() -> { try { - // jitter to avoid overloading database - long interval = missingShardDiscoveryInterval.roundTo(SECONDS); - SECONDS.sleep(ThreadLocalRandom.current().nextLong(1, interval)); + SECONDS.sleep(ThreadLocalRandom.current().nextInt(1, 30)); for (ShardMetadata shard : getMissingShards()) { stats.incrementBackgroundShardRecovery(); Futures.addCallback( diff --git a/presto-raptor/src/main/java/com/facebook/presto/raptor/util/DatabaseUtil.java b/presto-raptor/src/main/java/com/facebook/presto/raptor/util/DatabaseUtil.java index 6c11ee08af8d..99dafc5aba62 100644 --- a/presto-raptor/src/main/java/com/facebook/presto/raptor/util/DatabaseUtil.java +++ b/presto-raptor/src/main/java/com/facebook/presto/raptor/util/DatabaseUtil.java @@ -90,9 +90,15 @@ public static void runIgnoringConstraintViolation(Runnable task) task.run(); } catch (RuntimeException e) { - if (!sqlCodeStartsWith(e, "23")) { - throw e; + for (Throwable throwable : Throwables.getCausalChain(e)) { + if (throwable instanceof SQLException) { + String state = ((SQLException) throwable).getSQLState(); + if (state != null && state.startsWith("23")) { + return; + } + } } + throw e; } } @@ -128,22 +134,4 @@ public static void bindOptionalInt(PreparedStatement statement, int index, Optio statement.setNull(index, INTEGER); } } - - public static boolean isSyntaxOrAccessError(Exception e) - { - return sqlCodeStartsWith(e, "42"); - } - - private static boolean sqlCodeStartsWith(Exception e, String code) - { - for (Throwable throwable : Throwables.getCausalChain(e)) { - if (throwable instanceof SQLException) { - String state = ((SQLException) throwable).getSQLState(); - if (state != null && state.startsWith(code)) { - return true; - } - } - } - return false; - } } diff --git a/presto-raptor/src/test/java/com/facebook/presto/raptor/RaptorBenchmarkQueryRunner.java b/presto-raptor/src/test/java/com/facebook/presto/raptor/RaptorBenchmarkQueryRunner.java index 84d1a04ffe82..20ffd2ffd01f 100644 --- a/presto-raptor/src/test/java/com/facebook/presto/raptor/RaptorBenchmarkQueryRunner.java +++ b/presto-raptor/src/test/java/com/facebook/presto/raptor/RaptorBenchmarkQueryRunner.java @@ -54,7 +54,7 @@ public static LocalQueryRunner createLocalQueryRunner() LocalQueryRunner localQueryRunner = new LocalQueryRunner(session); // add tpch - localQueryRunner.createCatalog("tpch", new TpchConnectorFactory(1), ImmutableMap.of()); + localQueryRunner.createCatalog("tpch", new TpchConnectorFactory(1), ImmutableMap.of()); // add raptor ConnectorFactory raptorConnectorFactory = getOnlyElement(new RaptorPlugin() diff --git a/presto-raptor/src/test/java/com/facebook/presto/raptor/TestRaptorBucketFunction.java b/presto-raptor/src/test/java/com/facebook/presto/raptor/TestRaptorBucketFunction.java deleted file mode 100644 index 4a5cb73bcf2d..000000000000 --- a/presto-raptor/src/test/java/com/facebook/presto/raptor/TestRaptorBucketFunction.java +++ /dev/null @@ -1,92 +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.raptor; - -import com.facebook.presto.spi.BucketFunction; -import com.facebook.presto.spi.Page; -import com.facebook.presto.spi.block.Block; -import com.facebook.presto.spi.type.Type; -import com.google.common.collect.ImmutableList; -import org.testng.annotations.Test; - -import static com.facebook.presto.block.BlockAssertions.createIntsBlock; -import static com.facebook.presto.block.BlockAssertions.createLongsBlock; -import static com.facebook.presto.block.BlockAssertions.createStringsBlock; -import static com.facebook.presto.spi.type.BigintType.BIGINT; -import static com.facebook.presto.spi.type.IntegerType.INTEGER; -import static com.facebook.presto.spi.type.VarcharType.createUnboundedVarcharType; -import static org.testng.Assert.assertEquals; - -public class TestRaptorBucketFunction -{ - @Test - public void testBigint() - throws Exception - { - BucketFunction function = bucketFunction(50, BIGINT); - assertEquals(getBucket(function, createLongsBlock(123456789012L)), 12); - assertEquals(getBucket(function, createLongsBlock(454345325)), 16); - assertEquals(getBucket(function, createLongsBlock(365363)), 42); - assertEquals(getBucket(function, createLongsBlock(45645747)), 41); - assertEquals(getBucket(function, createLongsBlock(3244)), 29); - - function = bucketFunction(2, BIGINT); - assertEquals(getBucket(function, createLongsBlock(123456789012L)), 0); - assertEquals(getBucket(function, createLongsBlock(454345325)), 0); - assertEquals(getBucket(function, createLongsBlock(365363)), 0); - assertEquals(getBucket(function, createLongsBlock(45645747)), 1); - assertEquals(getBucket(function, createLongsBlock(3244)), 1); - } - - @Test - public void testInteger() - { - BucketFunction function = bucketFunction(50, INTEGER); - assertEquals(getBucket(function, createIntsBlock(454345325)), 16); - assertEquals(getBucket(function, createIntsBlock(365363)), 42); - assertEquals(getBucket(function, createIntsBlock(45645747)), 41); - assertEquals(getBucket(function, createIntsBlock(3244)), 29); - } - - @Test - public void testVarchar() - { - BucketFunction function = bucketFunction(50, createUnboundedVarcharType()); - assertEquals(getBucket(function, createStringsBlock("lorem ipsum")), 2); - assertEquals(getBucket(function, createStringsBlock("lorem")), 26); - assertEquals(getBucket(function, createStringsBlock("ipsum")), 3); - assertEquals(getBucket(function, createStringsBlock("hello")), 19); - } - - @Test - public void testVarcharBigint() - { - BucketFunction function = bucketFunction(50, createUnboundedVarcharType(), BIGINT); - assertEquals(getBucket(function, createStringsBlock("lorem ipsum"), createLongsBlock(123456789012L)), 24); - assertEquals(getBucket(function, createStringsBlock("lorem"), createLongsBlock(454345325)), 32); - assertEquals(getBucket(function, createStringsBlock("ipsum"), createLongsBlock(365363)), 21); - assertEquals(getBucket(function, createStringsBlock("hello"), createLongsBlock(45645747)), 34); - assertEquals(getBucket(function, createStringsBlock("world"), createLongsBlock(3244)), 4); - } - - private static int getBucket(BucketFunction function, Block... blocks) - { - return function.getBucket(new Page(blocks), 0); - } - - private static BucketFunction bucketFunction(int bucketCount, Type... types) - { - return new RaptorBucketFunction(bucketCount, ImmutableList.copyOf(types)); - } -} diff --git a/presto-raptor/src/test/java/com/facebook/presto/raptor/TestRaptorIntegrationSmokeTest.java b/presto-raptor/src/test/java/com/facebook/presto/raptor/TestRaptorIntegrationSmokeTest.java index 77d15d53a9c5..5e0c76a23bcd 100644 --- a/presto-raptor/src/test/java/com/facebook/presto/raptor/TestRaptorIntegrationSmokeTest.java +++ b/presto-raptor/src/test/java/com/facebook/presto/raptor/TestRaptorIntegrationSmokeTest.java @@ -362,21 +362,6 @@ public void testCreateBucketedTableLike() assertUpdate("DROP TABLE orders_bucketed_like"); } - @Test - public void testBucketingMixedTypes() - { - assertUpdate("" + - "CREATE TABLE orders_bucketed_mixed " + - "WITH (bucket_count = 50, bucketed_on = ARRAY ['custkey', 'clerk', 'shippriority']) " + - "AS SELECT * FROM orders", - "SELECT count(*) FROM orders"); - - assertQuery("SELECT * FROM orders_bucketed_mixed", "SELECT * FROM orders"); - assertQuery("SELECT count(*) FROM orders_bucketed_mixed", "SELECT count(*) FROM orders"); - assertQuery("SELECT count(DISTINCT \"$shard_uuid\") FROM orders_bucketed_mixed", "SELECT 50"); - assertQuery("SELECT count(DISTINCT \"$bucket_number\") FROM orders_bucketed_mixed", "SELECT 50"); - } - @Test public void testShowCreateTable() throws Exception diff --git a/presto-raptor/src/test/java/com/facebook/presto/raptor/metadata/TestDatabaseShardManager.java b/presto-raptor/src/test/java/com/facebook/presto/raptor/metadata/TestDatabaseShardManager.java index dbf966edccf1..22b51584336a 100644 --- a/presto-raptor/src/test/java/com/facebook/presto/raptor/metadata/TestDatabaseShardManager.java +++ b/presto-raptor/src/test/java/com/facebook/presto/raptor/metadata/TestDatabaseShardManager.java @@ -46,9 +46,6 @@ import java.io.File; import java.net.URI; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Statement; import java.time.LocalDate; import java.time.ZonedDateTime; import java.util.Collection; @@ -63,7 +60,6 @@ import java.util.UUID; import static com.facebook.presto.raptor.RaptorErrorCode.RAPTOR_EXTERNAL_BATCH_ALREADY_EXISTS; -import static com.facebook.presto.raptor.metadata.DatabaseShardManager.shardIndexTable; import static com.facebook.presto.raptor.storage.ShardStats.MAX_BINARY_INDEX_SIZE; import static com.facebook.presto.spi.StandardErrorCode.SERVER_STARTING_UP; import static com.facebook.presto.spi.StandardErrorCode.TRANSACTION_CONFLICT; @@ -83,7 +79,6 @@ import static com.google.common.collect.Iterators.concat; import static com.google.common.collect.Iterators.transform; import static io.airlift.slice.Slices.utf8Slice; -import static java.lang.String.format; import static java.time.ZoneOffset.UTC; import static java.util.concurrent.TimeUnit.DAYS; import static java.util.stream.Collectors.toSet; @@ -638,37 +633,6 @@ public void testShardPruningNoStats() shardAssertion(tableId).equal(c1, BIGINT, 3L).expected(shards); } - @Test - public void testAddNewColumn() - throws Exception - { - long tableId = createTable("test"); - List columns = ImmutableList.of(new ColumnInfo(1, BIGINT)); - shardManager.createTable(tableId, columns, false, OptionalLong.empty()); - int before = columnCount(tableId); - - ColumnInfo newColumn = new ColumnInfo(2, BIGINT); - shardManager.addColumn(tableId, newColumn); - int after = columnCount(tableId); - // should be 2 more: min and max columns - assertEquals(after, before + 2); - } - - @Test - public void testAddDuplicateColumn() - throws Exception - { - long tableId = createTable("test"); - List columns = ImmutableList.of(new ColumnInfo(1, BIGINT)); - shardManager.createTable(tableId, columns, false, OptionalLong.empty()); - int before = columnCount(tableId); - - shardManager.addColumn(tableId, columns.get(0)); - int after = columnCount(tableId); - // no error, no columns added - assertEquals(after, before); - } - private Set getShardNodes(long tableId, TupleDomain predicate) { try (ResultIterator iterator = shardManager.getShardNodes(tableId, predicate)) { @@ -783,14 +747,4 @@ private static Node createTestingNode() { return new PrestoNode(UUID.randomUUID().toString(), URI.create("http://test"), NodeVersion.UNKNOWN, false); } - - private int columnCount(long tableId) - throws SQLException - { - try (Statement statement = dummyHandle.getConnection().createStatement()) { - try (ResultSet rs = statement.executeQuery(format("SELECT * FROM %s LIMIT 0", shardIndexTable(tableId)))) { - return rs.getMetaData().getColumnCount(); - } - } - } } diff --git a/presto-raptor/src/test/java/com/facebook/presto/raptor/metadata/TestRaptorMetadata.java b/presto-raptor/src/test/java/com/facebook/presto/raptor/metadata/TestRaptorMetadata.java index ae4ab3d81030..81447980a9ac 100644 --- a/presto-raptor/src/test/java/com/facebook/presto/raptor/metadata/TestRaptorMetadata.java +++ b/presto-raptor/src/test/java/com/facebook/presto/raptor/metadata/TestRaptorMetadata.java @@ -47,6 +47,7 @@ import com.google.common.collect.ImmutableSet; import com.google.common.io.ByteArrayDataOutput; import io.airlift.json.JsonCodec; +import io.airlift.slice.Slice; import org.skife.jdbi.v2.DBI; import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.util.BooleanMapper; @@ -663,7 +664,7 @@ public void testTransactionInsert() assertNull(transactionSuccessful(transactionId)); // commit insert - metadata.finishInsert(SESSION, insertHandle, ImmutableList.of()); + metadata.finishInsert(SESSION, insertHandle, ImmutableList.of()); assertTrue(transactionExists(transactionId)); assertTrue(transactionSuccessful(transactionId)); } diff --git a/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestOrcStorageManager.java b/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestOrcStorageManager.java index 06af55788f2b..7187ff024aa1 100644 --- a/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestOrcStorageManager.java +++ b/presto-raptor/src/test/java/com/facebook/presto/raptor/storage/TestOrcStorageManager.java @@ -166,7 +166,7 @@ public void testWriter() OrcStorageManager manager = createOrcStorageManager(); List columnIds = ImmutableList.of(3L, 7L); - List columnTypes = ImmutableList.of(BIGINT, createVarcharType(10)); + List columnTypes = ImmutableList.of(BIGINT, createVarcharType(10)); StoragePageSink sink = createStoragePageSink(manager, columnIds, columnTypes); List pages = rowPagesBuilder(columnTypes) @@ -238,7 +238,7 @@ public void testReader() OrcStorageManager manager = createOrcStorageManager(); List columnIds = ImmutableList.of(2L, 4L, 6L, 7L, 8L, 9L); - List columnTypes = ImmutableList.of(BIGINT, createVarcharType(10), VARBINARY, DATE, BOOLEAN, DOUBLE); + List columnTypes = ImmutableList.of(BIGINT, createVarcharType(10), VARBINARY, DATE, BOOLEAN, DOUBLE); byte[] bytes1 = octets(0x00, 0xFE, 0xFF); byte[] bytes3 = octets(0x01, 0x02, 0x19, 0x80); @@ -313,7 +313,7 @@ public void testRewriter() long transactionId = TRANSACTION_ID; List columnIds = ImmutableList.of(3L, 7L); - List columnTypes = ImmutableList.of(BIGINT, createVarcharType(10)); + List columnTypes = ImmutableList.of(BIGINT, createVarcharType(10)); // create file with 2 rows StoragePageSink sink = createStoragePageSink(manager, columnIds, columnTypes); @@ -363,7 +363,7 @@ public void testWriterRollback() OrcStorageManager manager = createOrcStorageManager(); List columnIds = ImmutableList.of(3L, 7L); - List columnTypes = ImmutableList.of(BIGINT, createVarcharType(10)); + List columnTypes = ImmutableList.of(BIGINT, createVarcharType(10)); StoragePageSink sink = createStoragePageSink(manager, columnIds, columnTypes); List pages = rowPagesBuilder(columnTypes) @@ -496,7 +496,7 @@ public void testMaxShardRows() OrcStorageManager manager = createOrcStorageManager(2, new DataSize(2, MEGABYTE)); List columnIds = ImmutableList.of(3L, 7L); - List columnTypes = ImmutableList.of(BIGINT, createVarcharType(10)); + List columnTypes = ImmutableList.of(BIGINT, createVarcharType(10)); StoragePageSink sink = createStoragePageSink(manager, columnIds, columnTypes); List pages = rowPagesBuilder(columnTypes) @@ -512,7 +512,7 @@ public void testMaxFileSize() throws Exception { List columnIds = ImmutableList.of(3L, 7L); - List columnTypes = ImmutableList.of(BIGINT, createVarcharType(5)); + List columnTypes = ImmutableList.of(BIGINT, createVarcharType(5)); List pages = rowPagesBuilder(columnTypes) .row(123L, "hello") diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/csv/CsvFieldDecoder.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/csv/CsvFieldDecoder.java index e272102c429e..4daf52faa68a 100644 --- a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/csv/CsvFieldDecoder.java +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/csv/CsvFieldDecoder.java @@ -38,7 +38,7 @@ public class CsvFieldDecoder @Override public Set> getJavaTypes() { - return ImmutableSet.of(boolean.class, long.class, double.class, Slice.class); + return ImmutableSet.>of(boolean.class, long.class, double.class, Slice.class); } @Override diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/dummy/DummyFieldDecoder.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/dummy/DummyFieldDecoder.java index 3f0d2353c580..41ccd8415a83 100644 --- a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/dummy/DummyFieldDecoder.java +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/dummy/DummyFieldDecoder.java @@ -35,7 +35,7 @@ public class DummyFieldDecoder @Override public Set> getJavaTypes() { - return ImmutableSet.of(boolean.class, long.class, double.class, Slice.class); + return ImmutableSet.>of(boolean.class, long.class, double.class, Slice.class); } @Override diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/CustomDateTimeJsonFieldDecoder.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/CustomDateTimeJsonFieldDecoder.java index dd1d62b43f23..b2be7ce49347 100644 --- a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/CustomDateTimeJsonFieldDecoder.java +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/CustomDateTimeJsonFieldDecoder.java @@ -39,7 +39,7 @@ public class CustomDateTimeJsonFieldDecoder @Override public Set> getJavaTypes() { - return ImmutableSet.of(long.class, Slice.class); + return ImmutableSet.>of(long.class, Slice.class); } @Override diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/ISO8601JsonFieldDecoder.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/ISO8601JsonFieldDecoder.java index 5b7dd0c45b68..fc8c20f32e73 100644 --- a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/ISO8601JsonFieldDecoder.java +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/ISO8601JsonFieldDecoder.java @@ -46,7 +46,7 @@ public class ISO8601JsonFieldDecoder @Override public Set> getJavaTypes() { - return ImmutableSet.of(long.class, Slice.class); + return ImmutableSet.>of(long.class, Slice.class); } @Override diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/JsonFieldDecoder.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/JsonFieldDecoder.java index f8852e066faf..60953c42c706 100644 --- a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/JsonFieldDecoder.java +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/JsonFieldDecoder.java @@ -48,7 +48,7 @@ public class JsonFieldDecoder @Override public Set> getJavaTypes() { - return ImmutableSet.of(boolean.class, long.class, double.class, Slice.class); + return ImmutableSet.>of(boolean.class, long.class, double.class, Slice.class); } @Override diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/MillisecondsSinceEpochJsonFieldDecoder.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/MillisecondsSinceEpochJsonFieldDecoder.java index 130fa0d6ad11..628aa983485f 100644 --- a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/MillisecondsSinceEpochJsonFieldDecoder.java +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/MillisecondsSinceEpochJsonFieldDecoder.java @@ -49,7 +49,7 @@ public class MillisecondsSinceEpochJsonFieldDecoder @Override public Set> getJavaTypes() { - return ImmutableSet.of(long.class, Slice.class); + return ImmutableSet.>of(long.class, Slice.class); } @Override diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/RFC2822JsonFieldDecoder.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/RFC2822JsonFieldDecoder.java index 079cf538b7c0..0c50944e8b37 100644 --- a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/RFC2822JsonFieldDecoder.java +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/RFC2822JsonFieldDecoder.java @@ -47,7 +47,7 @@ public class RFC2822JsonFieldDecoder @Override public Set> getJavaTypes() { - return ImmutableSet.of(long.class, Slice.class); + return ImmutableSet.>of(long.class, Slice.class); } @Override diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/SecondsSinceEpochJsonFieldDecoder.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/SecondsSinceEpochJsonFieldDecoder.java index f68b3aaabc74..e6988b3f1a36 100644 --- a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/SecondsSinceEpochJsonFieldDecoder.java +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/json/SecondsSinceEpochJsonFieldDecoder.java @@ -49,7 +49,7 @@ public class SecondsSinceEpochJsonFieldDecoder @Override public Set> getJavaTypes() { - return ImmutableSet.of(long.class, Slice.class); + return ImmutableSet.>of(long.class, Slice.class); } @Override diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/raw/RawFieldDecoder.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/raw/RawFieldDecoder.java index cfe212fba70f..b2d42d93d75a 100644 --- a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/raw/RawFieldDecoder.java +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/raw/RawFieldDecoder.java @@ -78,7 +78,7 @@ static FieldType forString(String value) @Override public Set> getJavaTypes() { - return ImmutableSet.of(boolean.class, long.class, double.class, Slice.class); + return ImmutableSet.>of(boolean.class, long.class, double.class, Slice.class); } @Override diff --git a/presto-redis/src/main/java/com/facebook/presto/redis/RedisMetadata.java b/presto-redis/src/main/java/com/facebook/presto/redis/RedisMetadata.java index eea5ddc16b12..96ffc8664edc 100644 --- a/presto-redis/src/main/java/com/facebook/presto/redis/RedisMetadata.java +++ b/presto-redis/src/main/java/com/facebook/presto/redis/RedisMetadata.java @@ -147,7 +147,7 @@ public ConnectorTableLayout getTableLayout(ConnectorSession session, ConnectorTa RedisTableLayoutHandle layout = convertLayout(handle); // tables in this connector have a single layout - return getTableLayouts(session, layout.getTable(), Constraint.alwaysTrue(), Optional.empty()) + return getTableLayouts(session, layout.getTable(), Constraint.alwaysTrue(), Optional.empty()) .get(0) .getTableLayout(); } diff --git a/presto-redis/src/main/java/com/facebook/presto/redis/RedisTableDescriptionSupplier.java b/presto-redis/src/main/java/com/facebook/presto/redis/RedisTableDescriptionSupplier.java index 8a66f6656964..0c33347d2142 100644 --- a/presto-redis/src/main/java/com/facebook/presto/redis/RedisTableDescriptionSupplier.java +++ b/presto-redis/src/main/java/com/facebook/presto/redis/RedisTableDescriptionSupplier.java @@ -88,8 +88,8 @@ public Map get() log.debug("Created dummy Table definition for %s", tableName); builder.put(tableName, new RedisTableDescription(tableName.getTableName(), tableName.getSchemaName(), - new RedisTableFieldGroup(DummyRowDecoder.NAME, null, ImmutableList.of()), - new RedisTableFieldGroup(DummyRowDecoder.NAME, null, ImmutableList.of()))); + new RedisTableFieldGroup(DummyRowDecoder.NAME, null, ImmutableList.of()), + new RedisTableFieldGroup(DummyRowDecoder.NAME, null, ImmutableList.of()))); } } diff --git a/presto-redis/src/main/java/com/facebook/presto/redis/decoder/hash/HashRedisFieldDecoder.java b/presto-redis/src/main/java/com/facebook/presto/redis/decoder/hash/HashRedisFieldDecoder.java index 1222de25d9d2..395e98caa57a 100644 --- a/presto-redis/src/main/java/com/facebook/presto/redis/decoder/hash/HashRedisFieldDecoder.java +++ b/presto-redis/src/main/java/com/facebook/presto/redis/decoder/hash/HashRedisFieldDecoder.java @@ -29,7 +29,7 @@ public class HashRedisFieldDecoder @Override public Set> getJavaTypes() { - return ImmutableSet.of(boolean.class, long.class, double.class, Slice.class); + return ImmutableSet.>of(boolean.class, long.class, double.class, Slice.class); } @Override diff --git a/presto-redis/src/test/java/com/facebook/presto/redis/util/CodecSupplier.java b/presto-redis/src/test/java/com/facebook/presto/redis/util/CodecSupplier.java index 007da3b45a51..33f200fd1d7d 100644 --- a/presto-redis/src/test/java/com/facebook/presto/redis/util/CodecSupplier.java +++ b/presto-redis/src/test/java/com/facebook/presto/redis/util/CodecSupplier.java @@ -16,6 +16,7 @@ import com.facebook.presto.metadata.Metadata; import com.facebook.presto.spi.type.Type; import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; import com.fasterxml.jackson.databind.deser.std.FromStringDeserializer; import com.google.common.collect.ImmutableMap; import io.airlift.json.JsonCodec; @@ -36,7 +37,7 @@ public CodecSupplier(Class clazz, Metadata metadata) { this.clazz = clazz; ObjectMapperProvider objectMapperProvider = new ObjectMapperProvider(); - objectMapperProvider.setJsonDeserializers(ImmutableMap.of(Type.class, new TypeDeserializer(metadata))); + objectMapperProvider.setJsonDeserializers(ImmutableMap., JsonDeserializer>of(Type.class, new TypeDeserializer(metadata))); this.codecFactory = new JsonCodecFactory(objectMapperProvider); } diff --git a/presto-resource-group-managers/src/main/java/com/facebook/presto/resourceGroups/AbstractResourceConfigurationManager.java b/presto-resource-group-managers/src/main/java/com/facebook/presto/resourceGroups/AbstractResourceConfigurationManager.java index ae28d3cb1050..84583314b3fc 100644 --- a/presto-resource-group-managers/src/main/java/com/facebook/presto/resourceGroups/AbstractResourceConfigurationManager.java +++ b/presto-resource-group-managers/src/main/java/com/facebook/presto/resourceGroups/AbstractResourceConfigurationManager.java @@ -70,46 +70,21 @@ protected List buildSelectors(ManagerSpec managerSpec) { ImmutableList.Builder selectors = ImmutableList.builder(); for (SelectorSpec spec : managerSpec.getSelectors()) { - validateSelectors(managerSpec.getRootGroups(), spec.getGroup().getSegments()); selectors.add(new StaticSelector(spec.getUserRegex(), spec.getSourceRegex(), spec.getGroup())); } return selectors.build(); } - private void validateSelectors(List groups, List selectorGroups) - { - StringBuilder fullyQualifiedGroupName = new StringBuilder(); - while (!selectorGroups.isEmpty()) { - ResourceGroupNameTemplate groupName = selectorGroups.get(0); - fullyQualifiedGroupName.append(groupName); - Optional match = groups - .stream() - .filter(groupSpec -> groupSpec.getName().equals(groupName)) - .findFirst(); - if (!match.isPresent()) { - throw new IllegalArgumentException(format("Selector refers to nonexistent group: %s", fullyQualifiedGroupName.toString())); - } - fullyQualifiedGroupName.append("."); - groups = match.get().getSubGroups(); - selectorGroups = selectorGroups.subList(1, selectorGroups.size()); - } - } - protected AbstractResourceConfigurationManager(ClusterMemoryPoolManager memoryPoolManager) { memoryPoolManager.addChangeListener(new MemoryPoolId("general"), poolInfo -> { - Map memoryLimits = new HashMap<>(); synchronized (generalPoolMemoryFraction) { for (Map.Entry entry : generalPoolMemoryFraction.entrySet()) { double bytes = poolInfo.getMaxBytes() * entry.getValue(); - // setSoftMemoryLimit() acquires a lock on the root group of its tree, which could cause a deadlock if done while holding the "generalPoolMemoryFraction" lock - memoryLimits.put(entry.getKey(), new DataSize(bytes, BYTE)); + entry.getKey().setSoftMemoryLimit(new DataSize(bytes, BYTE)); } generalPoolBytes = poolInfo.getMaxBytes(); } - for (Map.Entry entry : memoryLimits.entrySet()) { - entry.getKey().setSoftMemoryLimit(entry.getValue()); - } }); } diff --git a/presto-resource-group-managers/src/main/java/com/facebook/presto/resourceGroups/ResourceGroupIdTemplate.java b/presto-resource-group-managers/src/main/java/com/facebook/presto/resourceGroups/ResourceGroupIdTemplate.java index 03c8f18b0646..c3913e4c2dd1 100644 --- a/presto-resource-group-managers/src/main/java/com/facebook/presto/resourceGroups/ResourceGroupIdTemplate.java +++ b/presto-resource-group-managers/src/main/java/com/facebook/presto/resourceGroups/ResourceGroupIdTemplate.java @@ -18,7 +18,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.google.common.base.Joiner; import com.google.common.base.Splitter; -import com.google.common.collect.ImmutableList; import java.util.List; import java.util.Objects; @@ -67,11 +66,6 @@ public ResourceGroupId expandTemplate(SelectionContext context) return id; } - public List getSegments() - { - return ImmutableList.copyOf(segments); - } - @Override public String toString() { diff --git a/presto-resource-group-managers/src/main/java/com/facebook/presto/resourceGroups/db/ResourceGroupSpecBuilder.java b/presto-resource-group-managers/src/main/java/com/facebook/presto/resourceGroups/db/ResourceGroupSpecBuilder.java index 121c478fade8..43c576129d8a 100644 --- a/presto-resource-group-managers/src/main/java/com/facebook/presto/resourceGroups/db/ResourceGroupSpecBuilder.java +++ b/presto-resource-group-managers/src/main/java/com/facebook/presto/resourceGroups/db/ResourceGroupSpecBuilder.java @@ -39,7 +39,7 @@ public class ResourceGroupSpecBuilder private final Optional softCpuLimit; private final Optional hardCpuLimit; private final Optional parentId; - private final ImmutableList.Builder subGroups = ImmutableList.builder(); + private final ImmutableList.Builder subGroups = ImmutableList.builder(); ResourceGroupSpecBuilder( long id, diff --git a/presto-resource-group-managers/src/test/java/com/facebook/presto/resourceGroups/TestFileResourceGroupConfigurationManager.java b/presto-resource-group-managers/src/test/java/com/facebook/presto/resourceGroups/TestFileResourceGroupConfigurationManager.java index a7b85f1587d5..cf3c1f6c8848 100644 --- a/presto-resource-group-managers/src/test/java/com/facebook/presto/resourceGroups/TestFileResourceGroupConfigurationManager.java +++ b/presto-resource-group-managers/src/test/java/com/facebook/presto/resourceGroups/TestFileResourceGroupConfigurationManager.java @@ -78,12 +78,6 @@ public void testConfiguration() assertEquals(sub.getJmxExport(), false); } - @Test(expectedExceptions = IllegalArgumentException.class, expectedExceptionsMessageRegExp = "Selector refers to nonexistent group: a.b.c.X") - public void testNonExistentGroup() - { - parse("resource_groups_config_bad_selector.json"); - } - private FileResourceGroupConfigurationManager parse(String fileName) { FileResourceGroupConfig config = new FileResourceGroupConfig(); diff --git a/presto-resource-group-managers/src/test/resources/resource_groups_config_bad_selector.json b/presto-resource-group-managers/src/test/resources/resource_groups_config_bad_selector.json deleted file mode 100644 index 2548cadc601f..000000000000 --- a/presto-resource-group-managers/src/test/resources/resource_groups_config_bad_selector.json +++ /dev/null @@ -1,46 +0,0 @@ -{ - "rootGroups": [ - { - "name": "a", - "softMemoryLimit": "1MB", - "maxRunning": 100, - "maxQueued": 1000, - "subGroups": [ - { - "name": "b", - "softMemoryLimit": "1MB", - "maxRunning": 100, - "maxQueued": 1000, - "subGroups": [ - { - "name": "c", - "softMemoryLimit": "1MB", - "maxRunning": 100, - "maxQueued": 1000 - } - ] - } - ] - }, - { - "name": "A", - "softMemoryLimit": "1MB", - "maxRunning": 100, - "maxQueued": 1000 - } - ], - "selectors": [ - { - "group": "a.b" - }, - { - "group": "a.b.c" - }, - { - "group": "A" - }, - { - "group": "a.b.c.X" - } - ] -} diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/ColumnMetadata.java b/presto-spi/src/main/java/com/facebook/presto/spi/ColumnMetadata.java index afd5c6b7768d..a3fd6ca0e24e 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/ColumnMetadata.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/ColumnMetadata.java @@ -24,7 +24,6 @@ public class ColumnMetadata private final String name; private final Type type; private final String comment; - private final String extraInfo; private final boolean hidden; public ColumnMetadata(String name, Type type) @@ -33,11 +32,6 @@ public ColumnMetadata(String name, Type type) } public ColumnMetadata(String name, Type type, String comment, boolean hidden) - { - this(name, type, comment, null, hidden); - } - - public ColumnMetadata(String name, Type type, String comment, String extraInfo, boolean hidden) { if (name == null || name.isEmpty()) { throw new NullPointerException("name is null or empty"); @@ -49,7 +43,6 @@ public ColumnMetadata(String name, Type type, String comment, String extraInfo, this.name = name.toLowerCase(ENGLISH); this.type = type; this.comment = comment; - this.extraInfo = extraInfo; this.hidden = hidden; } @@ -68,11 +61,6 @@ public String getComment() return comment; } - public String getExtraInfo() - { - return extraInfo; - } - public boolean isHidden() { return hidden; @@ -87,9 +75,6 @@ public String toString() if (comment != null) { sb.append(", comment='").append(comment).append('\''); } - if (extraInfo != null) { - sb.append(", extraInfo='").append(extraInfo).append('\''); - } if (hidden) { sb.append(", hidden"); } @@ -100,7 +85,7 @@ public String toString() @Override public int hashCode() { - return Objects.hash(name, type, comment, extraInfo, hidden); + return Objects.hash(name, type, comment, hidden); } @Override @@ -116,7 +101,6 @@ public boolean equals(Object obj) return Objects.equals(this.name, other.name) && Objects.equals(this.type, other.type) && Objects.equals(this.comment, other.comment) && - Objects.equals(this.extraInfo, other.extraInfo) && Objects.equals(this.hidden, other.hidden); } } diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorPageSink.java b/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorPageSink.java index f715c254ff48..bf654f66c6ea 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorPageSink.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorPageSink.java @@ -29,13 +29,6 @@ public interface ConnectorPageSink */ CompletableFuture appendPage(Page page); - /** - * Notifies the connector that no more pages will be appended and returns - * connector-specific information that will be sent to the coordinator to - * complete the write operation. This method may be called immediately - * after the previous call to {@link #appendPage} (even if the returned - * future is not complete). - */ CompletableFuture> finish(); void abort(); diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorPageSource.java b/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorPageSource.java index 5e7f10511537..b49e1f5c9736 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorPageSource.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/ConnectorPageSource.java @@ -15,13 +15,10 @@ import java.io.Closeable; import java.io.IOException; -import java.util.concurrent.CompletableFuture; public interface ConnectorPageSource extends Closeable { - CompletableFuture NOT_BLOCKED = CompletableFuture.completedFuture(null); - /** * Gets the total input bytes that will be processed by this page source. * This is normally the same size as the split. If size is not available, @@ -65,14 +62,4 @@ public interface ConnectorPageSource @Override void close() throws IOException; - - /** - * Returns a future that will be completed when the page source becomes - * unblocked. If the page source is not blocked, this method should return - * {@code NOT_BLOCKED}. - */ - default CompletableFuture isBlocked() - { - return NOT_BLOCKED; - } } diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/DiscretePredicates.java b/presto-spi/src/main/java/com/facebook/presto/spi/DiscretePredicates.java index e835033a33f2..bbffac309d09 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/DiscretePredicates.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/DiscretePredicates.java @@ -24,17 +24,16 @@ public final class DiscretePredicates { private final List columns; - private final Iterable> predicates; + private final List> predicates; - public DiscretePredicates(List columns, Iterable> predicates) + public DiscretePredicates(List columns, List> predicates) { requireNonNull(columns, "columns is null"); if (columns.isEmpty()) { throw new IllegalArgumentException("columns is empty"); } this.columns = unmodifiableList(new ArrayList<>(columns)); - // do not copy predicates because it may be lazy - this.predicates = requireNonNull(predicates, "predicates is null"); + this.predicates = unmodifiableList(new ArrayList<>(requireNonNull(predicates, "predicates is null"))); } public List getColumns() @@ -42,7 +41,7 @@ public List getColumns() return columns; } - public Iterable> getPredicates() + public List> getPredicates() { return predicates; } diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/block/AbstractArrayBlock.java b/presto-spi/src/main/java/com/facebook/presto/spi/block/AbstractArrayBlock.java index c422f5c8c440..0fefcfe8c68d 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/block/AbstractArrayBlock.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/block/AbstractArrayBlock.java @@ -13,8 +13,11 @@ */ package com.facebook.presto.spi.block; +import io.airlift.slice.Slice; +import io.airlift.slice.SliceOutput; +import io.airlift.slice.Slices; + import java.util.ArrayList; -import java.util.Arrays; import java.util.List; public abstract class AbstractArrayBlock @@ -22,16 +25,11 @@ public abstract class AbstractArrayBlock { protected abstract Block getValues(); - protected abstract int[] getOffsets(); + protected abstract Slice getOffsets(); protected abstract int getOffsetBase(); - protected abstract boolean[] getValueIsNull(); - - private int getOffset(int position) - { - return getOffsets()[position + getOffsetBase()]; - } + protected abstract Slice getValueIsNull(); @Override public BlockEncoding getEncoding() @@ -39,58 +37,52 @@ public BlockEncoding getEncoding() return new ArrayBlockEncoding(getValues().getEncoding()); } + private int getOffset(int position) + { + return position == 0 ? 0 : getOffsets().getInt((position - 1) * 4) - getOffsetBase(); + } + @Override public Block copyPositions(List positions) { - int[] newOffsets = new int[positions.size() + 1]; - boolean[] newValueIsNull = new boolean[positions.size()]; + SliceOutput newOffsets = Slices.allocate(positions.size() * Integer.BYTES).getOutput(); + SliceOutput newValueIsNull = Slices.allocate(positions.size()).getOutput(); List valuesPositions = new ArrayList<>(); - int newPosition = 0; + int countNewOffset = 0; for (int position : positions) { if (isNull(position)) { - newValueIsNull[newPosition] = true; - newOffsets[newPosition + 1] = newOffsets[newPosition]; + newValueIsNull.appendByte(1); + newOffsets.appendInt(countNewOffset); } else { - int valuesStartOffset = getOffset(position); - int valuesEndOffset = getOffset(position + 1); - int valuesLength = valuesEndOffset - valuesStartOffset; - - newOffsets[newPosition + 1] = newOffsets[newPosition] + valuesLength; - - for (int elementIndex = valuesStartOffset; elementIndex < valuesEndOffset; elementIndex++) { - valuesPositions.add(elementIndex); + newValueIsNull.appendByte(0); + int positionStartOffset = getOffset(position); + int positionEndOffset = getOffset(position + 1); + countNewOffset += positionEndOffset - positionStartOffset; + newOffsets.appendInt(countNewOffset); + for (int j = positionStartOffset; j < positionEndOffset; j++) { + valuesPositions.add(j); } } - newPosition++; } Block newValues = getValues().copyPositions(valuesPositions); - return new ArrayBlock(positions.size(), newValueIsNull, newOffsets, newValues); + return new ArrayBlock(newValues, newOffsets.slice(), 0, newValueIsNull.slice()); } @Override public Block getRegion(int position, int length) { - int positionCount = getPositionCount(); - if (position < 0 || length < 0 || position + length > positionCount) { - throw new IndexOutOfBoundsException("Invalid position " + position + " in block with " + positionCount + " positions"); - } - - if (position == 0 && length == positionCount) { - return this; - } - - return new ArrayBlock( - position + getOffsetBase(), - length, - getValueIsNull(), - getOffsets(), - getValues()); + return getRegion(position, length, false); } @Override public Block copyRegion(int position, int length) + { + return getRegion(position, length, true); + } + + private Block getRegion(int position, int length, boolean compact) { int positionCount = getPositionCount(); if (position < 0 || length < 0 || position + length > positionCount) { @@ -99,16 +91,33 @@ public Block copyRegion(int position, int length) int startValueOffset = getOffset(position); int endValueOffset = getOffset(position + length); - Block newValues = getValues().copyRegion(startValueOffset, endValueOffset - startValueOffset); - - int[] newOffsets = new int[length + 1]; - for (int i = 1; i < newOffsets.length; i++) { - newOffsets[i] = getOffset(position + i) - startValueOffset; + Block newValues; + Slice newOffsets; + Slice newValueIsNull; + int newOffsetBase; + if (compact) { + newValues = getValues().copyRegion(startValueOffset, endValueOffset - startValueOffset); + int[] newOffsetsArray = new int[length]; + for (int i = 0; i < length; i++) { + newOffsetsArray[i] = getOffset(position + i + 1) - getOffset(position); + } + newOffsets = Slices.wrappedIntArray(newOffsetsArray); + newValueIsNull = Slices.copyOf(getValueIsNull(), position, length); + newOffsetBase = 0; } - - boolean[] newValueIsNull = Arrays.copyOfRange(getValueIsNull(), position + getOffsetBase(), position + getOffsetBase() + length); - - return new ArrayBlock(length, newValueIsNull, newOffsets, newValues); + else { + if (position == 0 && length == positionCount) { + // It is incorrect to pull up this `if` because child blocks may be compact-able when this if condition is satisfied + return this; + } + else { + newValues = getValues().getRegion(startValueOffset, endValueOffset - startValueOffset); + newOffsets = getOffsets().slice(position * 4, length * 4); + newValueIsNull = getValueIsNull().slice(position, length); + newOffsetBase = startValueOffset + getOffsetBase(); + } + } + return new ArrayBlock(newValues, newOffsets, newOffsetBase, newValueIsNull); } @Override @@ -154,21 +163,23 @@ public Block getSingleValueBlock(int position) checkReadablePosition(position); int startValueOffset = getOffset(position); - int valueLength = getOffset(position + 1) - startValueOffset; - Block newValues = getValues().copyRegion(startValueOffset, valueLength); - - return new ArrayBlock( - 1, - new boolean[] {isNull(position)}, - new int[] {0, valueLength}, - newValues); + int endValueOffset = getOffset(position + 1); + + Block newValues = getValues().copyRegion(startValueOffset, endValueOffset - startValueOffset); + + // rewrite offsets so that baseOffset is always zero + Slice newOffsets = Slices.wrappedIntArray(endValueOffset - startValueOffset); + + Slice newValueIsNull = Slices.copyOf(getValueIsNull(), position, 1); + + return new ArrayBlock(newValues, newOffsets, 0, newValueIsNull); } @Override public boolean isNull(int position) { checkReadablePosition(position); - return getValueIsNull()[position + getOffsetBase()]; + return getValueIsNull().getByte(position) != 0; } public T apply(ArrayBlockFunction function, int position) diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/block/ArrayBlock.java b/presto-spi/src/main/java/com/facebook/presto/spi/block/ArrayBlock.java index 934b37f403f8..947603a04aa2 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/block/ArrayBlock.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/block/ArrayBlock.java @@ -13,10 +13,9 @@ */ package com.facebook.presto.spi.block; +import io.airlift.slice.Slice; import org.openjdk.jol.info.ClassLayout; -import static com.facebook.presto.spi.block.BlockUtil.intSaturatedCast; -import static io.airlift.slice.SizeOf.sizeOf; import static java.util.Objects.requireNonNull; public class ArrayBlock @@ -24,66 +23,35 @@ public class ArrayBlock { private static final int INSTANCE_SIZE = ClassLayout.parseClass(ArrayBlock.class).instanceSize(); - private final int arrayOffset; - private final int positionCount; - private final boolean[] valueIsNull; private final Block values; - private final int[] offsets; + private final Slice offsets; + private final int offsetBase; + private final Slice valueIsNull; - private final int sizeInBytes; - private final int retainedSizeInBytes; - - public ArrayBlock(int positionCount, boolean[] valueIsNull, int[] offsets, Block values) + public ArrayBlock(Block values, Slice offsets, int offsetBase, Slice valueIsNull) { - this(0, positionCount, valueIsNull, offsets, values); - } - - ArrayBlock(int arrayOffset, int positionCount, boolean[] valueIsNull, int[] offsets, Block values) - { - if (arrayOffset < 0) { - throw new IllegalArgumentException("arrayOffset is negative"); - } - this.arrayOffset = arrayOffset; - - if (positionCount < 0) { - throw new IllegalArgumentException("positionCount is negative"); - } - this.positionCount = positionCount; - - requireNonNull(valueIsNull, "valueIsNull is null"); - if (valueIsNull.length - arrayOffset < positionCount) { - throw new IllegalArgumentException("isNull length is less than positionCount"); - } - this.valueIsNull = valueIsNull; - - requireNonNull(offsets, "offsets is null"); - if (offsets.length - arrayOffset < positionCount + 1) { - throw new IllegalArgumentException("offsets length is less than positionCount"); - } - this.offsets = offsets; - this.values = requireNonNull(values); - - sizeInBytes = values.getSizeInBytes() + ((Integer.BYTES + Byte.BYTES) * this.positionCount); - retainedSizeInBytes = intSaturatedCast(INSTANCE_SIZE + values.getRetainedSizeInBytes() + sizeOf(offsets) + sizeOf(valueIsNull)); + this.offsets = requireNonNull(offsets); + this.offsetBase = offsetBase; + this.valueIsNull = requireNonNull(valueIsNull); } @Override public int getPositionCount() { - return positionCount; + return valueIsNull.length(); } @Override public int getSizeInBytes() { - return sizeInBytes; + return getValues().getSizeInBytes() + offsets.length() + valueIsNull.length(); } @Override public int getRetainedSizeInBytes() { - return retainedSizeInBytes; + return INSTANCE_SIZE + values.getRetainedSizeInBytes() + offsets.getRetainedSize() + valueIsNull.getRetainedSize(); } @Override @@ -93,7 +61,7 @@ protected Block getValues() } @Override - protected int[] getOffsets() + protected Slice getOffsets() { return offsets; } @@ -101,11 +69,11 @@ protected int[] getOffsets() @Override protected int getOffsetBase() { - return arrayOffset; + return offsetBase; } @Override - protected boolean[] getValueIsNull() + protected Slice getValueIsNull() { return valueIsNull; } diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/block/ArrayBlockBuilder.java b/presto-spi/src/main/java/com/facebook/presto/spi/block/ArrayBlockBuilder.java index 36af24378b6b..e5c590fc3c69 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/block/ArrayBlockBuilder.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/block/ArrayBlockBuilder.java @@ -14,13 +14,12 @@ package com.facebook.presto.spi.block; import com.facebook.presto.spi.type.Type; +import io.airlift.slice.DynamicSliceOutput; +import io.airlift.slice.Slice; +import io.airlift.slice.SliceOutput; import org.openjdk.jol.info.ClassLayout; -import java.util.Arrays; - import static com.facebook.presto.spi.block.BlockUtil.calculateBlockResetSize; -import static com.facebook.presto.spi.block.BlockUtil.intSaturatedCast; -import static io.airlift.slice.SizeOf.sizeOf; import static java.util.Objects.requireNonNull; public class ArrayBlockBuilder @@ -29,18 +28,13 @@ public class ArrayBlockBuilder { private static final int INSTANCE_SIZE = ClassLayout.parseClass(ArrayBlockBuilder.class).instanceSize() + BlockBuilderStatus.INSTANCE_SIZE; - private int positionCount; - private BlockBuilderStatus blockBuilderStatus; - - private int[] offsets; - private boolean[] valueIsNull; - private final BlockBuilder values; + private SliceOutput offsets; + private SliceOutput valueIsNull; + private static final int OFFSET_BASE = 0; private int currentEntrySize; - private int retainedSizeInBytes; - /** * Caller of this constructor is responsible for making sure `valuesBlock` is constructed with the same `blockBuilderStatus` as the one in the argument */ @@ -49,8 +43,8 @@ public ArrayBlockBuilder(BlockBuilder valuesBlock, BlockBuilderStatus blockBuild this( blockBuilderStatus, valuesBlock, - new int[expectedEntries + 1], - new boolean[expectedEntries]); + new DynamicSliceOutput(expectedEntries * Integer.BYTES), + new DynamicSliceOutput(expectedEntries)); } public ArrayBlockBuilder(Type elementType, BlockBuilderStatus blockBuilderStatus, int expectedEntries, int expectedBytesPerEntry) @@ -58,8 +52,8 @@ public ArrayBlockBuilder(Type elementType, BlockBuilderStatus blockBuilderStatus this( blockBuilderStatus, elementType.createBlockBuilder(blockBuilderStatus, expectedEntries, expectedBytesPerEntry), - new int[expectedEntries + 1], - new boolean[expectedEntries]); + new DynamicSliceOutput(expectedEntries * Integer.BYTES), + new DynamicSliceOutput(expectedEntries)); } public ArrayBlockBuilder(Type elementType, BlockBuilderStatus blockBuilderStatus, int expectedEntries) @@ -67,42 +61,37 @@ public ArrayBlockBuilder(Type elementType, BlockBuilderStatus blockBuilderStatus this( blockBuilderStatus, elementType.createBlockBuilder(blockBuilderStatus, expectedEntries), - new int[expectedEntries + 1], - new boolean[expectedEntries]); + new DynamicSliceOutput(expectedEntries * Integer.BYTES), + new DynamicSliceOutput(expectedEntries)); } /** * Caller of this private constructor is responsible for making sure `values` is constructed with the same `blockBuilderStatus` as the one in the argument */ - private ArrayBlockBuilder(BlockBuilderStatus blockBuilderStatus, BlockBuilder values, int[] offsets, boolean[] valueIsNull) + private ArrayBlockBuilder(BlockBuilderStatus blockBuilderStatus, BlockBuilder values, SliceOutput offsets, SliceOutput valueIsNull) { this.blockBuilderStatus = requireNonNull(blockBuilderStatus, "blockBuilderStatus is null"); this.values = requireNonNull(values, "values is null"); this.offsets = requireNonNull(offsets, "offset is null"); - this.valueIsNull = requireNonNull(valueIsNull, "valueIsNull is null"); - if (offsets.length != valueIsNull.length + 1) { - throw new IllegalArgumentException("expected offsets and valueIsNull to have same length"); - } - - updateDataSize(); + this.valueIsNull = requireNonNull(valueIsNull); } @Override public int getPositionCount() { - return positionCount; + return valueIsNull.size(); } @Override public int getSizeInBytes() { - return values.getSizeInBytes() + ((Integer.BYTES + Byte.BYTES) * positionCount); + return values.getSizeInBytes() + offsets.size() + valueIsNull.size(); } @Override public int getRetainedSizeInBytes() { - return retainedSizeInBytes; + return INSTANCE_SIZE + values.getRetainedSizeInBytes() + offsets.getRetainedSize() + valueIsNull.getRetainedSize(); } @Override @@ -112,21 +101,21 @@ protected Block getValues() } @Override - protected int[] getOffsets() + protected Slice getOffsets() { - return offsets; + return offsets.getUnderlyingSlice(); } @Override protected int getOffsetBase() { - return 0; + return OFFSET_BASE; } @Override - protected boolean[] getValueIsNull() + protected Slice getValueIsNull() { - return valueIsNull; + return valueIsNull.getUnderlyingSlice(); } @Override @@ -186,36 +175,19 @@ public BlockBuilder appendNull() private void entryAdded(boolean isNull) { - if (valueIsNull.length <= positionCount) { - growCapacity(); - } - offsets[positionCount + 1] = values.getPositionCount(); - valueIsNull[positionCount] = isNull; - positionCount++; + offsets.appendInt(values.getPositionCount()); + valueIsNull.appendByte(isNull ? 1 : 0); blockBuilderStatus.addBytes(Integer.BYTES + Byte.BYTES); } - private void growCapacity() - { - int newSize = BlockUtil.calculateNewArraySize(valueIsNull.length); - valueIsNull = Arrays.copyOf(valueIsNull, newSize); - offsets = Arrays.copyOf(offsets, newSize + 1); - updateDataSize(); - } - - private void updateDataSize() - { - retainedSizeInBytes = intSaturatedCast(INSTANCE_SIZE + sizeOf(valueIsNull) + sizeOf(offsets)); - } - @Override public ArrayBlock build() { if (currentEntrySize > 0) { throw new IllegalStateException("Current entry must be closed before the block can be built"); } - return new ArrayBlock(positionCount, valueIsNull, offsets, values.build()); + return new ArrayBlock(values.build(), offsets.slice(), OFFSET_BASE, valueIsNull.slice()); } @Override @@ -224,14 +196,11 @@ public void reset(BlockBuilderStatus blockBuilderStatus) this.blockBuilderStatus = requireNonNull(blockBuilderStatus, "blockBuilderStatus is null"); int newSize = calculateBlockResetSize(getPositionCount()); - valueIsNull = new boolean[newSize]; - offsets = new int[newSize + 1]; + valueIsNull = new DynamicSliceOutput(newSize); + offsets = new DynamicSliceOutput(newSize * Integer.BYTES); values.reset(blockBuilderStatus); currentEntrySize = 0; - positionCount = 0; - - updateDataSize(); } @Override diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/block/ArrayBlockEncoding.java b/presto-spi/src/main/java/com/facebook/presto/spi/block/ArrayBlockEncoding.java index 93e4f2b91421..544ba01d9886 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/block/ArrayBlockEncoding.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/block/ArrayBlockEncoding.java @@ -42,20 +42,11 @@ public void writeBlock(SliceOutput sliceOutput, Block block) { AbstractArrayBlock arrayBlock = (AbstractArrayBlock) block; + valueBlockEncoding.writeBlock(sliceOutput, arrayBlock.getValues()); + sliceOutput.appendInt(arrayBlock.getOffsetBase()); int positionCount = arrayBlock.getPositionCount(); - - int offsetBase = arrayBlock.getOffsetBase(); - int[] offsets = arrayBlock.getOffsets(); - - int valuesStartOffset = offsets[offsetBase]; - int valuesEndOffset = offsets[offsetBase + positionCount]; - Block values = arrayBlock.getValues().getRegion(valuesStartOffset, valuesEndOffset - valuesStartOffset); - valueBlockEncoding.writeBlock(sliceOutput, values); - sliceOutput.appendInt(positionCount); - for (int position = 0; position < positionCount + 1; position++) { - sliceOutput.writeInt(offsets[offsetBase + position] - valuesStartOffset); - } + sliceOutput.writeBytes(arrayBlock.getOffsets(), 0, positionCount * 4); EncoderUtil.encodeNullsAsBits(sliceOutput, block); } @@ -63,12 +54,12 @@ public void writeBlock(SliceOutput sliceOutput, Block block) public Block readBlock(SliceInput sliceInput) { Block values = valueBlockEncoding.readBlock(sliceInput); - + int offsetBase = sliceInput.readInt(); int positionCount = sliceInput.readInt(); - int[] offsets = new int[positionCount + 1]; - sliceInput.readBytes(Slices.wrappedIntArray(offsets)); + byte[] offsets = new byte[positionCount * 4]; + sliceInput.readBytes(offsets); boolean[] valueIsNull = EncoderUtil.decodeNullBits(sliceInput, positionCount); - return new ArrayBlock(positionCount, valueIsNull, offsets, values); + return new ArrayBlock(values, Slices.wrappedBuffer(offsets), offsetBase, Slices.wrappedBooleanArray(valueIsNull)); } @Override diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/predicate/SortedRangeSet.java b/presto-spi/src/main/java/com/facebook/presto/spi/predicate/SortedRangeSet.java index d158841a3702..dfa50d3c9c47 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/predicate/SortedRangeSet.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/predicate/SortedRangeSet.java @@ -60,7 +60,7 @@ private SortedRangeSet(Type type, NavigableMap lowIndexedRanges) static SortedRangeSet none(Type type) { - return copyOf(type, Collections.emptyList()); + return copyOf(type, Collections.emptyList()); } static SortedRangeSet all(Type type) diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/type/AbstractType.java b/presto-spi/src/main/java/com/facebook/presto/spi/type/AbstractType.java index 2c8c9b01092a..486c4132211a 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/type/AbstractType.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/type/AbstractType.java @@ -90,67 +90,67 @@ public int compareTo(Block leftBlock, int leftPosition, Block rightBlock, int ri @Override public boolean getBoolean(Block block, int position) { - throw new UnsupportedOperationException(getClass().getName()); + throw new UnsupportedOperationException(); } @Override public void writeBoolean(BlockBuilder blockBuilder, boolean value) { - throw new UnsupportedOperationException(getClass().getName()); + throw new UnsupportedOperationException(); } @Override public long getLong(Block block, int position) { - throw new UnsupportedOperationException(getClass().getName()); + throw new UnsupportedOperationException(); } @Override public void writeLong(BlockBuilder blockBuilder, long value) { - throw new UnsupportedOperationException(getClass().getName()); + throw new UnsupportedOperationException(); } @Override public double getDouble(Block block, int position) { - throw new UnsupportedOperationException(getClass().getName()); + throw new UnsupportedOperationException(); } @Override public void writeDouble(BlockBuilder blockBuilder, double value) { - throw new UnsupportedOperationException(getClass().getName()); + throw new UnsupportedOperationException(); } @Override public Slice getSlice(Block block, int position) { - throw new UnsupportedOperationException(getClass().getName()); + throw new UnsupportedOperationException(); } @Override public void writeSlice(BlockBuilder blockBuilder, Slice value) { - throw new UnsupportedOperationException(getClass().getName()); + throw new UnsupportedOperationException(); } @Override public void writeSlice(BlockBuilder blockBuilder, Slice value, int offset, int length) { - throw new UnsupportedOperationException(getClass().getName()); + throw new UnsupportedOperationException(); } @Override public Object getObject(Block block, int position) { - throw new UnsupportedOperationException(getClass().getName()); + throw new UnsupportedOperationException(); } @Override public void writeObject(BlockBuilder blockBuilder, Object value) { - throw new UnsupportedOperationException(getClass().getName()); + throw new UnsupportedOperationException(); } @Override diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/type/CharType.java b/presto-spi/src/main/java/com/facebook/presto/spi/type/CharType.java index 0b153d0b1587..5a41103983d3 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/type/CharType.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/type/CharType.java @@ -23,7 +23,6 @@ import java.util.Objects; import static com.facebook.presto.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; -import static com.facebook.presto.spi.type.Chars.compareChars; import static java.lang.String.format; import static java.util.Collections.singletonList; @@ -107,10 +106,9 @@ public long hash(Block block, int position) @Override public int compareTo(Block leftBlock, int leftPosition, Block rightBlock, int rightPosition) { - Slice leftSlice = leftBlock.getSlice(leftPosition, 0, leftBlock.getLength(leftPosition)); - Slice rightSlice = rightBlock.getSlice(rightPosition, 0, rightBlock.getLength(rightPosition)); - - return compareChars(leftSlice, rightSlice); + int leftLength = leftBlock.getLength(leftPosition); + int rightLength = rightBlock.getLength(rightPosition); + return leftBlock.compareTo(leftPosition, 0, leftLength, rightBlock, rightPosition, 0, rightLength); } @Override diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/type/Chars.java b/presto-spi/src/main/java/com/facebook/presto/spi/type/Chars.java index 275e6e76b3af..81151de1fbfc 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/type/Chars.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/type/Chars.java @@ -112,42 +112,4 @@ private static int sliceLengthWithoutTrailingSpaces(Slice slice) } return 0; } - - public static int compareChars(Slice left, Slice right) - { - if (left.length() < right.length()) { - return compareCharsShorterToLonger(left, right); - } - else { - return -compareCharsShorterToLonger(right, left); - } - } - - private static int compareCharsShorterToLonger(Slice shorter, Slice longer) - { - for (int i = 0; i < shorter.length(); ++i) { - int result = compareUnsignedBytes(shorter.getByte(i), longer.getByte(i)); - if (result != 0) { - return result; - } - } - - for (int i = shorter.length(); i < longer.length(); ++i) { - int result = compareUnsignedBytes((byte) ' ', longer.getByte(i)); - if (result != 0) { - return result; - } - } - return 0; - } - - private static int compareUnsignedBytes(byte thisByte, byte thatByte) - { - return unsignedByteToInt(thisByte) - unsignedByteToInt(thatByte); - } - - private static int unsignedByteToInt(byte thisByte) - { - return thisByte & 0xFF; - } } diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/type/Decimals.java b/presto-spi/src/main/java/com/facebook/presto/spi/type/Decimals.java index b9ba1edc8979..4a5be7310cf6 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/type/Decimals.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/type/Decimals.java @@ -30,10 +30,10 @@ import static java.lang.Math.pow; import static java.lang.Math.round; import static java.lang.String.format; +import static java.math.BigDecimal.ROUND_UNNECESSARY; import static java.math.BigInteger.TEN; -import static java.math.RoundingMode.UNNECESSARY; -public final class Decimals +public class Decimals { private Decimals() {} @@ -132,7 +132,6 @@ private static String getMatcherGroup(Matcher matcher, int group) return groupValue; } - @SuppressWarnings("NumericCastThatLosesPrecision") public static Slice encodeUnscaledValue(BigInteger unscaledValue) { Slice result = Slices.allocate(SIZE_OF_LONG_DECIMAL); @@ -256,7 +255,7 @@ public static void writeBigDecimal(DecimalType decimalType, BlockBuilder blockBu public static BigDecimal rescale(BigDecimal value, DecimalType type) { - value = value.setScale(type.getScale(), UNNECESSARY); + value = value.setScale(type.getScale(), ROUND_UNNECESSARY); if (value.precision() > type.getPrecision()) { throw new IllegalArgumentException("decimal precision larger than column precision"); diff --git a/presto-spi/src/test/java/com/facebook/presto/spi/predicate/TestTupleDomain.java b/presto-spi/src/test/java/com/facebook/presto/spi/predicate/TestTupleDomain.java index 0f16bd4b93e3..38e88e024008 100644 --- a/presto-spi/src/test/java/com/facebook/presto/spi/predicate/TestTupleDomain.java +++ b/presto-spi/src/test/java/com/facebook/presto/spi/predicate/TestTupleDomain.java @@ -217,15 +217,15 @@ public void testOverlaps() throws Exception { assertTrue(overlaps( - ImmutableMap.of(), - ImmutableMap.of())); + ImmutableMap.of(), + ImmutableMap.of())); assertTrue(overlaps( - ImmutableMap.of(), + ImmutableMap.of(), ImmutableMap.of(A, Domain.singleValue(BIGINT, 0L)))); assertFalse(overlaps( - ImmutableMap.of(), + ImmutableMap.of(), ImmutableMap.of(A, Domain.none(BIGINT)))); assertFalse(overlaps( @@ -270,24 +270,24 @@ public void testContains() throws Exception { assertTrue(contains( - ImmutableMap.of(), - ImmutableMap.of())); + ImmutableMap.of(), + ImmutableMap.of())); assertTrue(contains( - ImmutableMap.of(), + ImmutableMap.of(), ImmutableMap.of(A, Domain.none(BIGINT)))); assertTrue(contains( - ImmutableMap.of(), + ImmutableMap.of(), ImmutableMap.of(A, Domain.all(BIGINT)))); assertTrue(contains( - ImmutableMap.of(), + ImmutableMap.of(), ImmutableMap.of(A, Domain.singleValue(DOUBLE, 0.0)))); assertFalse(contains( ImmutableMap.of(A, Domain.none(BIGINT)), - ImmutableMap.of())); + ImmutableMap.of())); assertTrue(contains( ImmutableMap.of(A, Domain.none(BIGINT)), @@ -303,7 +303,7 @@ public void testContains() assertTrue(contains( ImmutableMap.of(A, Domain.all(BIGINT)), - ImmutableMap.of())); + ImmutableMap.of())); assertTrue(contains( ImmutableMap.of(A, Domain.all(BIGINT)), @@ -319,7 +319,7 @@ public void testContains() assertFalse(contains( ImmutableMap.of(A, Domain.singleValue(BIGINT, 0L)), - ImmutableMap.of())); + ImmutableMap.of())); assertTrue(contains( ImmutableMap.of(A, Domain.singleValue(BIGINT, 0L)), @@ -413,19 +413,19 @@ public void testEquals() throws Exception { assertTrue(equals( - ImmutableMap.of(), - ImmutableMap.of())); + ImmutableMap.of(), + ImmutableMap.of())); assertTrue(equals( - ImmutableMap.of(), + ImmutableMap.of(), ImmutableMap.of(A, Domain.all(BIGINT)))); assertFalse(equals( - ImmutableMap.of(), + ImmutableMap.of(), ImmutableMap.of(A, Domain.none(BIGINT)))); assertFalse(equals( - ImmutableMap.of(), + ImmutableMap.of(), ImmutableMap.of(A, Domain.singleValue(BIGINT, 0L)))); assertTrue(equals( diff --git a/presto-spi/src/test/java/com/facebook/presto/spi/type/TestTypeSignature.java b/presto-spi/src/test/java/com/facebook/presto/spi/type/TestTypeSignature.java index 883733414822..0f8e1edb8c63 100644 --- a/presto-spi/src/test/java/com/facebook/presto/spi/type/TestTypeSignature.java +++ b/presto-spi/src/test/java/com/facebook/presto/spi/type/TestTypeSignature.java @@ -146,9 +146,9 @@ private TypeSignature signature(String name) public void parseSignature() throws Exception { - assertSignature("bigint", "bigint", ImmutableList.of()); - assertSignature("boolean", "boolean", ImmutableList.of()); - assertSignature("varchar", "varchar", ImmutableList.of()); + assertSignature("bigint", "bigint", ImmutableList.of()); + assertSignature("boolean", "boolean", ImmutableList.of()); + assertSignature("varchar", "varchar", ImmutableList.of()); assertSignature("array(bigint)", "array", ImmutableList.of("bigint")); assertSignature("array(array(bigint))", "array", ImmutableList.of("array(bigint)")); diff --git a/presto-spi/src/test/java/com/facebook/presto/spi/type/TestingTypeManager.java b/presto-spi/src/test/java/com/facebook/presto/spi/type/TestingTypeManager.java index a4118054596e..2561ffc93b85 100644 --- a/presto-spi/src/test/java/com/facebook/presto/spi/type/TestingTypeManager.java +++ b/presto-spi/src/test/java/com/facebook/presto/spi/type/TestingTypeManager.java @@ -51,7 +51,7 @@ public Type getParameterizedType(String baseTypeName, List getTypes() { - return ImmutableList.of(BOOLEAN, BIGINT, DOUBLE, VARCHAR, VARBINARY, TIMESTAMP, DATE, ID, HYPER_LOG_LOG); + return ImmutableList.of(BOOLEAN, BIGINT, DOUBLE, VARCHAR, VARBINARY, TIMESTAMP, DATE, ID, HYPER_LOG_LOG); } @Override diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 2d8d6e714669..1dd7f0988428 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -199,32 +199,13 @@ org.apache.maven.plugins maven-surefire-plugin - **/TestDistributedQueriesNoHashGeneration.java **/TestLocalQueries.java **/TestLocalQueriesIndexed.java - **/TestLocalBinarySpilledQueries.java - - - - ci - - - - org.apache.maven.plugins - maven-surefire-plugin - - - - - - - - diff --git a/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestDistributedQueries.java b/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestDistributedQueries.java index 893f029861e9..0b92bf18ce1f 100644 --- a/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestDistributedQueries.java +++ b/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestDistributedQueries.java @@ -687,9 +687,9 @@ public void testViewMetadata() // test SHOW COLUMNS actual = computeActual("SHOW COLUMNS FROM meta_test_view"); - expected = resultBuilder(getSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) - .row("x", "bigint", "", "") - .row("y", "varchar(3)", "", "") + expected = resultBuilder(getSession(), VARCHAR, VARCHAR, VARCHAR) + .row("x", "bigint", "") + .row("y", "varchar(3)", "") .build(); assertEquals(actual, expected); diff --git a/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestIntegrationSmokeTest.java b/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestIntegrationSmokeTest.java index f794f60147ef..51a0cb4b23e7 100644 --- a/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestIntegrationSmokeTest.java +++ b/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestIntegrationSmokeTest.java @@ -146,29 +146,29 @@ private MaterializedResult getExpectedTableDescription(boolean dateSupported, bo orderDateType = "varchar"; } if (parametrizedVarchar) { - return MaterializedResult.resultBuilder(queryRunner.getDefaultSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) - .row("orderkey", "bigint", "", "") - .row("custkey", "bigint", "", "") - .row("orderstatus", "varchar", "", "") - .row("totalprice", "double", "", "") - .row("orderdate", orderDateType, "", "") - .row("orderpriority", "varchar", "", "") - .row("clerk", "varchar", "", "") - .row("shippriority", "integer", "", "") - .row("comment", "varchar", "", "") + return MaterializedResult.resultBuilder(queryRunner.getDefaultSession(), VARCHAR, VARCHAR, VARCHAR) + .row("orderkey", "bigint", "") + .row("custkey", "bigint", "") + .row("orderstatus", "varchar", "") + .row("totalprice", "double", "") + .row("orderdate", orderDateType, "") + .row("orderpriority", "varchar", "") + .row("clerk", "varchar", "") + .row("shippriority", "integer", "") + .row("comment", "varchar", "") .build(); } else { - return MaterializedResult.resultBuilder(queryRunner.getDefaultSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) - .row("orderkey", "bigint", "", "") - .row("custkey", "bigint", "", "") - .row("orderstatus", "varchar(1)", "", "") - .row("totalprice", "double", "", "") - .row("orderdate", orderDateType, "", "") - .row("orderpriority", "varchar(15)", "", "") - .row("clerk", "varchar(15)", "", "") - .row("shippriority", "integer", "", "") - .row("comment", "varchar(79)", "", "") + return MaterializedResult.resultBuilder(queryRunner.getDefaultSession(), VARCHAR, VARCHAR, VARCHAR) + .row("orderkey", "bigint", "") + .row("custkey", "bigint", "") + .row("orderstatus", "varchar(1)", "") + .row("totalprice", "double", "") + .row("orderdate", orderDateType, "") + .row("orderpriority", "varchar(15)", "") + .row("clerk", "varchar(15)", "") + .row("shippriority", "integer", "") + .row("comment", "varchar(79)", "") .build(); } } diff --git a/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestQueries.java b/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestQueries.java index 328fc3917ea6..c82c8f9176eb 100644 --- a/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestQueries.java +++ b/presto-tests/src/main/java/com/facebook/presto/tests/AbstractTestQueries.java @@ -51,7 +51,6 @@ import java.util.stream.IntStream; import static com.facebook.presto.connector.informationSchema.InformationSchemaMetadata.INFORMATION_SCHEMA; -import static com.facebook.presto.operator.scalar.ApplyFunction.APPLY_FUNCTION; import static com.facebook.presto.spi.type.BigintType.BIGINT; import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; import static com.facebook.presto.spi.type.DateType.DATE; @@ -97,7 +96,6 @@ public abstract class AbstractTestQueries .window(CustomRank.class) .scalars(CustomAdd.class) .scalars(CreateHll.class) - .function(APPLY_FUNCTION) .getFunctions(); public static final List> TEST_SYSTEM_PROPERTIES = ImmutableList.of( @@ -169,39 +167,6 @@ public void testLimitIntMax() assertQuery("SELECT orderkey from orders ORDER BY orderkey LIMIT " + Integer.MAX_VALUE); } - @Test - public void testNonDeterministic() - { - MaterializedResult materializedResult = computeActual("SELECT rand() FROM orders LIMIT 10"); - long distinctCount = materializedResult.getMaterializedRows().stream() - .map(row -> row.getField(0)) - .distinct() - .count(); - assertTrue(distinctCount >= 8, "rand() must produce different rows"); - - materializedResult = computeActual("SELECT apply(1, x -> x + rand()) FROM orders LIMIT 10"); - distinctCount = materializedResult.getMaterializedRows().stream() - .map(row -> row.getField(0)) - .distinct() - .count(); - assertTrue(distinctCount >= 8, "rand() must produce different rows"); - } - - @Test - public void testLambdaInAggregationContext() - { - assertQuery("SELECT apply(sum(x), i -> i * i) FROM (VALUES 1, 2, 3, 4, 5) t(x)", "SELECT 225"); - assertQuery("SELECT apply(x, i -> i - 1), sum(y) FROM (VALUES (1, 10), (1, 20), (2, 50)) t(x,y) group by x", "VALUES (0, 30), (1, 50)"); - assertQuery("SELECT x, apply(sum(y), i -> i * 10) FROM (VALUES (1, 10), (1, 20), (2, 50)) t(x,y) group by x", "VALUES (1, 300), (2, 500)"); - } - - @Test - public void testLambdaInSubqueryContext() - { - assertQuery("SELECT apply(x, i -> i * i) FROM (SELECT 10 x)", "SELECT 100"); - assertQuery("SELECT apply((SELECT 10), i -> i * i)", "SELECT 100"); - } - @Test public void testNonDeterministicFilter() { @@ -992,35 +957,6 @@ public void testOrderByExpressionWithLimit() assertQueryOrdered("SELECT custkey, orderstatus FROM ORDERS ORDER BY orderkey + 1 DESC LIMIT 10"); } - @Test - public void testOrderByWithOutputColumnReference() - { - assertQueryOrdered("SELECT a*2 AS b FROM (VALUES -1, 0, 2) t(a) ORDER BY b*-1", "VALUES 4, 0, -2"); - assertQueryOrdered("SELECT a*2 AS b FROM (VALUES -1, 0, 2) t(a) ORDER BY b", "VALUES -2, 0, 4"); - assertQueryOrdered("SELECT a*-2 AS a FROM (VALUES -1, 0, 2) t(a) ORDER BY a*-1", "VALUES 2, 0, -4"); - assertQueryOrdered("SELECT a*-2 AS a FROM (VALUES -1, 0, 2) t(a) ORDER BY t.a*-1", "VALUES -4, 0, 2"); - assertQueryOrdered("SELECT a*-2 FROM (VALUES -1, 0, 2) t(a) ORDER BY a*-1", "VALUES -4, 0, 2"); - assertQueryOrdered("SELECT a*-2 FROM (VALUES -1, 0, 2) t(a) ORDER BY t.a*-1", "VALUES -4, 0, 2"); - assertQueryOrdered("SELECT a, a* -1 AS a FROM (VALUES -1, 0, 2) t(a) ORDER BY t.a", "VALUES (-1, 1), (0, 0), (2, -2)"); - assertQueryOrdered("SELECT a, a* -2 AS b FROM (VALUES -1, 0, 2) t(a) ORDER BY a + b", "VALUES (2, -4), (0, 0), (-1, 2)"); - assertQueryOrdered("SELECT a as b, a* -2 AS a FROM (VALUES -1, 0, 2) t(a) ORDER BY a + b", "VALUES (2, -4), (0, 0), (-1, 2)"); - assertQueryOrdered("SELECT a* -2 AS a FROM (VALUES -1, 0, 2) t(a) ORDER BY a + t.a", "VALUES -4, 0, 2"); - - assertQueryFails("SELECT a, a* -1 AS a FROM (VALUES -1, 0, 2) t(a) ORDER BY a", ".*'a' is ambiguous"); - } - - @Test - public void testOrderByWithAggregation() - throws Exception - { - assertQuery("" + - "SELECT x, sum(cast(x AS double))\n" + - "FROM (VALUES '1.0') t(x)\n" + - "GROUP BY x\n" + - "ORDER BY sum(cast(x AS double))", - "VALUES ('1.0', 1.0)"); - } - @Test public void testGroupByOrderByLimit() { @@ -1723,7 +1659,6 @@ public void testIntersect() "SELECT * FROM (VALUES 1, 2) " + "INTERSECT SELECT * FROM (VALUES 1.0, 2)", "VALUES 1.0, 2.0"); - assertQuery("SELECT NULL, NULL INTERSECT SELECT NULL, NULL FROM nation"); MaterializedResult emptyResult = computeActual("SELECT 100 INTERSECT (SELECT regionkey FROM nation WHERE nationkey <10)"); assertEquals(emptyResult.getMaterializedRows().size(), 0); @@ -1782,11 +1717,6 @@ public void testExcept() assertQuery( "SELECT * FROM (VALUES 1, 2) " + "EXCEPT SELECT * FROM (VALUES 3.0, 2)"); - assertQuery("SELECT NULL, NULL EXCEPT SELECT NULL, NULL FROM nation"); - - assertQuery( - "(SELECT * FROM (VALUES 1) EXCEPT SELECT * FROM (VALUES 0))" + - "EXCEPT (SELECT * FROM (VALUES 1) EXCEPT SELECT * FROM (VALUES 1))"); MaterializedResult emptyResult = computeActual("SELECT 0 EXCEPT (SELECT regionkey FROM nation WHERE nationkey <10)"); assertEquals(emptyResult.getMaterializedRows().size(), 0); @@ -2732,9 +2662,6 @@ public void testInUncorrelatedSubquery() assertQuery( "SELECT x FROM (VALUES 2) t(x) where MAP(ARRAY[8589934592], ARRAY[x]) IN (VALUES MAP(ARRAY[8589934592],ARRAY[2]))", "SELECT 2"); - assertQuery( - "SELECT a IN (VALUES 2), a FROM (VALUES (2)) t(a)", - "SELECT TRUE, 2"); } @Test @@ -3261,6 +3188,13 @@ public void testOrderByMultipleFields() assertQueryOrdered("SELECT custkey, orderstatus FROM orders ORDER BY custkey DESC, orderstatus"); } + @Test + public void testOrderByDuplicateFields() + { + assertQueryOrdered("SELECT custkey, custkey FROM orders ORDER BY custkey, custkey"); + assertQueryOrdered("SELECT custkey, custkey FROM orders ORDER BY custkey ASC, custkey DESC"); + } + @Test public void testOrderByWithNulls() { @@ -5001,28 +4935,28 @@ public void testShowColumns() { MaterializedResult actual = computeActual("SHOW COLUMNS FROM orders"); - MaterializedResult expectedUnparametrizedVarchar = resultBuilder(getSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) - .row("orderkey", "bigint", "", "") - .row("custkey", "bigint", "", "") - .row("orderstatus", "varchar", "", "") - .row("totalprice", "double", "", "") - .row("orderdate", "date", "", "") - .row("orderpriority", "varchar", "", "") - .row("clerk", "varchar", "", "") - .row("shippriority", "integer", "", "") - .row("comment", "varchar", "", "") + MaterializedResult expectedUnparametrizedVarchar = resultBuilder(getSession(), VARCHAR, VARCHAR, VARCHAR) + .row("orderkey", "bigint", "") + .row("custkey", "bigint", "") + .row("orderstatus", "varchar", "") + .row("totalprice", "double", "") + .row("orderdate", "date", "") + .row("orderpriority", "varchar", "") + .row("clerk", "varchar", "") + .row("shippriority", "integer", "") + .row("comment", "varchar", "") .build(); - MaterializedResult expectedParametrizedVarchar = resultBuilder(getSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) - .row("orderkey", "bigint", "", "") - .row("custkey", "bigint", "", "") - .row("orderstatus", "varchar(1)", "", "") - .row("totalprice", "double", "", "") - .row("orderdate", "date", "", "") - .row("orderpriority", "varchar(15)", "", "") - .row("clerk", "varchar(15)", "", "") - .row("shippriority", "integer", "", "") - .row("comment", "varchar(79)", "", "") + MaterializedResult expectedParametrizedVarchar = resultBuilder(getSession(), VARCHAR, VARCHAR, VARCHAR) + .row("orderkey", "bigint", "") + .row("custkey", "bigint", "") + .row("orderstatus", "varchar(1)", "") + .row("totalprice", "double", "") + .row("orderdate", "date", "") + .row("orderpriority", "varchar(15)", "") + .row("clerk", "varchar(15)", "") + .row("shippriority", "integer", "") + .row("comment", "varchar(79)", "") .build(); // Until we migrate all connectors to parametrized varchar we check two options @@ -5272,7 +5206,6 @@ public void testUnion() assertQuery("SELECT orderkey FROM orders UNION SELECT custkey FROM orders"); assertQuery("SELECT 123 UNION DISTINCT SELECT 123 UNION ALL SELECT 123"); assertQuery("SELECT NULL UNION SELECT NULL"); - assertQuery("SELECT NULL, NULL UNION ALL SELECT NULL, NULL FROM nation"); // mixed single-node vs fixed vs source-distributed assertQuery("SELECT orderkey FROM orders UNION ALL SELECT 123 UNION ALL (SELECT custkey FROM orders GROUP BY custkey)"); @@ -5658,9 +5591,6 @@ public void testSemiJoin() "SELECT * FROM (VALUES (1,1), (2,2), (3, 3)) t(x, y) WHERE (x+y in (VALUES 4, 5)) AND (x*y in (VALUES 4, 5))", "VALUES (2,2)"); - // test multi level IN subqueries - assertQuery("SELECT 1 IN (SELECT 1), 2 IN (SELECT 1) WHERE 1 IN (SELECT 1)"); - // Throw in a bunch of IN subquery predicates assertQuery("" + "SELECT *, o2.custkey\n" + @@ -6842,7 +6772,7 @@ public void testTypeMismatch() @Test public void testInvalidType() { - assertQueryFails("SELECT CAST(null AS array(foo))", "\\Qline 1:8: Unknown type: array(foo)\\E"); + assertQueryFails("SELECT CAST(null AS array(foo))", "\\Qline 1:8: Unknown type: ARRAY(FOO)\\E"); } @Test @@ -7519,18 +7449,6 @@ public void testExecuteUsing() "VALUES (2, 4), (3, 8), (4, 4)"); } - @Test - public void testExecuteUsingComplexJoinCriteria() - { - String query = "SELECT * FROM (VALUES 1) t(a) JOIN (VALUES 2) u(a) ON t.a + u.a < ?"; - Session session = Session.builder(getSession()) - .addPreparedStatement("my_query", query) - .build(); - assertQuery(session, - "EXECUTE my_query USING 5", - "VALUES (1, 2)"); - } - @Test public void testExecuteUsingWithSubquery() { @@ -7619,19 +7537,6 @@ public void testDescribeInput() assertEqualsIgnoreOrder(actual, expected); } - @Test - public void testDescribeInputWithAggregation() - { - Session session = Session.builder(getSession()) - .addPreparedStatement("my_query", "select count(*) + ? from nation") - .build(); - MaterializedResult actual = computeActual(session, "DESCRIBE INPUT my_query"); - MaterializedResult expected = resultBuilder(session, BIGINT, VARCHAR) - .row(0, "bigint") - .build(); - assertEqualsIgnoreOrder(actual, expected); - } - @Test public void testDescribeInputNoParameters() { @@ -7709,37 +7614,10 @@ public void testDescribeOutputNamedAndUnnamed() } @Test - public void testDescribeOutputNonSelect() - { - assertDescribeOutputRowCount("CREATE TABLE foo AS SELECT * FROM nation"); - assertDescribeOutputRowCount("DELETE FROM orders"); - - assertDescribeOutputEmpty("CALL foo()"); - assertDescribeOutputEmpty("SET SESSION optimize_hash_generation=false"); - assertDescribeOutputEmpty("RESET SESSION optimize_hash_generation"); - assertDescribeOutputEmpty("START TRANSACTION"); - assertDescribeOutputEmpty("COMMIT"); - assertDescribeOutputEmpty("ROLLBACK"); - assertDescribeOutputEmpty("GRANT INSERT ON foo TO bar"); - assertDescribeOutputEmpty("REVOKE INSERT ON foo FROM bar"); - assertDescribeOutputEmpty("CREATE SCHEMA foo"); - assertDescribeOutputEmpty("ALTER SCHEMA foo RENAME TO bar"); - assertDescribeOutputEmpty("DROP SCHEMA foo"); - assertDescribeOutputEmpty("CREATE TABLE foo (x bigint)"); - assertDescribeOutputEmpty("ALTER TABLE foo ADD COLUMN y bigint"); - assertDescribeOutputEmpty("ALTER TABLE foo RENAME TO bar"); - assertDescribeOutputEmpty("DROP TABLE foo"); - assertDescribeOutputEmpty("CREATE VIEW foo AS SELECT * FROM nation"); - assertDescribeOutputEmpty("DROP VIEW foo"); - assertDescribeOutputEmpty("PREPARE test FROM SELECT * FROM orders"); - assertDescribeOutputEmpty("EXECUTE test"); - assertDescribeOutputEmpty("DEALLOCATE PREPARE test"); - } - - private void assertDescribeOutputRowCount(@Language("SQL") String sql) + public void testDescribeOutputRowCountQuery() { Session session = Session.builder(getSession()) - .addPreparedStatement("my_query", sql) + .addPreparedStatement("my_query", "CREATE TABLE foo AS SELECT * FROM nation") .build(); MaterializedResult actual = computeActual(session, "DESCRIBE OUTPUT my_query"); @@ -7749,10 +7627,11 @@ private void assertDescribeOutputRowCount(@Language("SQL") String sql) assertEqualsIgnoreOrder(actual, expected); } - private void assertDescribeOutputEmpty(@Language("SQL") String sql) + @Test + public void testDescribeOutputDataDefinitionQuery() { Session session = Session.builder(getSession()) - .addPreparedStatement("my_query", sql) + .addPreparedStatement("my_query", "SET SESSION optimize_hash_generation=false") .build(); MaterializedResult actual = computeActual(session, "DESCRIBE OUTPUT my_query"); diff --git a/presto-tests/src/main/java/com/facebook/presto/tests/DistributedQueryRunner.java b/presto-tests/src/main/java/com/facebook/presto/tests/DistributedQueryRunner.java index 0cf29a92f6d7..8b115ef4ccad 100644 --- a/presto-tests/src/main/java/com/facebook/presto/tests/DistributedQueryRunner.java +++ b/presto-tests/src/main/java/com/facebook/presto/tests/DistributedQueryRunner.java @@ -253,7 +253,7 @@ public void installPlugin(Plugin plugin) public void createCatalog(String catalogName, String connectorName) { - createCatalog(catalogName, connectorName, ImmutableMap.of()); + createCatalog(catalogName, connectorName, ImmutableMap.of()); } @Override diff --git a/presto-tests/src/main/java/com/facebook/presto/tests/StandaloneQueryRunner.java b/presto-tests/src/main/java/com/facebook/presto/tests/StandaloneQueryRunner.java index e10b0b3465fa..6de8a5b9d563 100644 --- a/presto-tests/src/main/java/com/facebook/presto/tests/StandaloneQueryRunner.java +++ b/presto-tests/src/main/java/com/facebook/presto/tests/StandaloneQueryRunner.java @@ -184,7 +184,7 @@ public void installPlugin(Plugin plugin) public void createCatalog(String catalogName, String connectorName) { - createCatalog(catalogName, connectorName, ImmutableMap.of()); + createCatalog(catalogName, connectorName, ImmutableMap.of()); } public void createCatalog(String catalogName, String connectorName, Map properties) diff --git a/presto-tests/src/main/java/com/facebook/presto/tests/tpch/TpchIndexSpec.java b/presto-tests/src/main/java/com/facebook/presto/tests/tpch/TpchIndexSpec.java index ca9c8ba07ef0..e4eae91be11d 100644 --- a/presto-tests/src/main/java/com/facebook/presto/tests/tpch/TpchIndexSpec.java +++ b/presto-tests/src/main/java/com/facebook/presto/tests/tpch/TpchIndexSpec.java @@ -22,7 +22,7 @@ public class TpchIndexSpec { - public static final TpchIndexSpec NO_INDEXES = new TpchIndexSpec(ImmutableSetMultimap.of()); + public static final TpchIndexSpec NO_INDEXES = new TpchIndexSpec(ImmutableSetMultimap.>of()); private final SetMultimap> spec; diff --git a/presto-tests/src/test/java/com/facebook/presto/tests/TestLocalBinarySpilledQueries.java b/presto-tests/src/test/java/com/facebook/presto/tests/TestLocalBinarySpilledQueries.java deleted file mode 100644 index ffa027e191ce..000000000000 --- a/presto-tests/src/test/java/com/facebook/presto/tests/TestLocalBinarySpilledQueries.java +++ /dev/null @@ -1,62 +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.tests; - -import com.facebook.presto.Session; -import com.facebook.presto.SystemSessionProperties; -import com.facebook.presto.connector.ConnectorId; -import com.facebook.presto.metadata.SessionPropertyManager; -import com.facebook.presto.testing.LocalQueryRunner; -import com.facebook.presto.tpch.TpchConnectorFactory; -import com.google.common.collect.ImmutableMap; - -import static com.facebook.presto.testing.TestingSession.TESTING_CATALOG; -import static com.facebook.presto.testing.TestingSession.testSessionBuilder; -import static com.facebook.presto.tpch.TpchMetadata.TINY_SCHEMA_NAME; - -public class TestLocalBinarySpilledQueries - extends AbstractTestQueries -{ - public TestLocalBinarySpilledQueries() - { - super(createLocalQueryRunner()); - } - - private static LocalQueryRunner createLocalQueryRunner() - { - Session defaultSession = testSessionBuilder() - .setCatalog("local") - .setSchema(TINY_SCHEMA_NAME) - .setSystemProperty(SystemSessionProperties.SPILL_ENABLED, "true") - .setSystemProperty(SystemSessionProperties.OPERATOR_MEMORY_LIMIT_BEFORE_SPILL, "1B") //spill constantly - .build(); - - LocalQueryRunner localQueryRunner = new LocalQueryRunner(defaultSession); - - // add the tpch catalog - // local queries run directly against the generator - localQueryRunner.createCatalog( - defaultSession.getCatalog().get(), - new TpchConnectorFactory(1), - ImmutableMap.of()); - - localQueryRunner.getMetadata().addFunctions(CUSTOM_FUNCTIONS); - - SessionPropertyManager sessionPropertyManager = localQueryRunner.getMetadata().getSessionPropertyManager(); - sessionPropertyManager.addSystemSessionProperties(TEST_SYSTEM_PROPERTIES); - sessionPropertyManager.addConnectorSessionProperties(new ConnectorId(TESTING_CATALOG), TEST_CATALOG_PROPERTIES); - - return localQueryRunner; - } -} diff --git a/presto-tests/src/test/java/com/facebook/presto/tests/TestLocalQueries.java b/presto-tests/src/test/java/com/facebook/presto/tests/TestLocalQueries.java index 0ad40c20faaf..93e5eccc6030 100644 --- a/presto-tests/src/test/java/com/facebook/presto/tests/TestLocalQueries.java +++ b/presto-tests/src/test/java/com/facebook/presto/tests/TestLocalQueries.java @@ -46,7 +46,7 @@ public static LocalQueryRunner createLocalQueryRunner() localQueryRunner.createCatalog( defaultSession.getCatalog().get(), new TpchConnectorFactory(1), - ImmutableMap.of()); + ImmutableMap.of()); localQueryRunner.getMetadata().addFunctions(CUSTOM_FUNCTIONS); diff --git a/presto-tests/src/test/java/com/facebook/presto/tests/TestLocalQueriesIndexed.java b/presto-tests/src/test/java/com/facebook/presto/tests/TestLocalQueriesIndexed.java index 0103f83e56de..6ef1110cd1a0 100644 --- a/presto-tests/src/test/java/com/facebook/presto/tests/TestLocalQueriesIndexed.java +++ b/presto-tests/src/test/java/com/facebook/presto/tests/TestLocalQueriesIndexed.java @@ -41,7 +41,7 @@ private static LocalQueryRunner createLocalQueryRunner() // add the tpch catalog // local queries run directly against the generator localQueryRunner.createCatalog(defaultSession.getCatalog().get(), - new IndexedTpchConnectorFactory(INDEX_SPEC, 1), ImmutableMap.of()); + new IndexedTpchConnectorFactory(INDEX_SPEC, 1), ImmutableMap.of()); return localQueryRunner; } diff --git a/presto-tpch/src/main/java/com/facebook/presto/tpch/TpchMetadata.java b/presto-tpch/src/main/java/com/facebook/presto/tpch/TpchMetadata.java index 6c7879996cc0..42507acadf0b 100644 --- a/presto-tpch/src/main/java/com/facebook/presto/tpch/TpchMetadata.java +++ b/presto-tpch/src/main/java/com/facebook/presto/tpch/TpchMetadata.java @@ -142,8 +142,8 @@ else if (tableHandle.getTableName().equals(TpchTable.LINE_ITEM.getTableName())) ConnectorTableLayout layout = new ConnectorTableLayout( new TpchTableLayoutHandle(tableHandle), - Optional.empty(), - TupleDomain.all(), // TODO: return well-known properties (e.g., orderkey > 0, etc) + Optional.>empty(), + TupleDomain.all(), // TODO: return well-known properties (e.g., orderkey > 0, etc) nodePartition, partitioningColumns, Optional.empty(), @@ -158,7 +158,7 @@ public ConnectorTableLayout getTableLayout(ConnectorSession session, ConnectorTa TpchTableLayoutHandle layout = checkType(handle, TpchTableLayoutHandle.class, "layout"); // tables in this connector have a single layout - return getTableLayouts(session, layout.getTable(), Constraint.alwaysTrue(), Optional.empty()) + return getTableLayouts(session, layout.getTable(), Constraint.alwaysTrue(), Optional.empty()) .get(0) .getTableLayout(); } diff --git a/presto-verifier/src/main/java/com/facebook/presto/verifier/Validator.java b/presto-verifier/src/main/java/com/facebook/presto/verifier/Validator.java index 668e232c25cd..a25b31b36cc8 100644 --- a/presto-verifier/src/main/java/com/facebook/presto/verifier/Validator.java +++ b/presto-verifier/src/main/java/com/facebook/presto/verifier/Validator.java @@ -206,12 +206,12 @@ private boolean validate() // query has too many rows. Consider blacklisting. if (controlResult.getState() == State.TOO_MANY_ROWS) { - testResult = new QueryResult(State.INVALID, null, null, null, null, ImmutableList.of()); + testResult = new QueryResult(State.INVALID, null, null, null, null, ImmutableList.>of()); return false; } // query failed in the control if (controlResult.getState() != State.SUCCESS) { - testResult = new QueryResult(State.INVALID, null, null, null, null, ImmutableList.of()); + testResult = new QueryResult(State.INVALID, null, null, null, null, ImmutableList.>of()); return true; } @@ -239,7 +239,7 @@ private static QueryResult tearDown(Query query, List postQueryResu QueryResult queryResult = executor.apply(postqueryString); postQueryResults.add(queryResult); if (queryResult.getState() != State.SUCCESS) { - return new QueryResult(State.FAILED_TO_TEARDOWN, queryResult.getException(), queryResult.getWallTime(), queryResult.getCpuTime(), queryResult.getQueryId(), ImmutableList.of()); + return new QueryResult(State.FAILED_TO_TEARDOWN, queryResult.getException(), queryResult.getWallTime(), queryResult.getCpuTime(), queryResult.getQueryId(), ImmutableList.>of()); } } @@ -253,7 +253,7 @@ private static QueryResult setup(Query query, List preQueryResults, QueryResult queryResult = executor.apply(prequeryString); preQueryResults.add(queryResult); if (queryResult.getState() != State.SUCCESS) { - return new QueryResult(State.FAILED_TO_SETUP, queryResult.getException(), queryResult.getWallTime(), queryResult.getCpuTime(), queryResult.getQueryId(), ImmutableList.of()); + return new QueryResult(State.FAILED_TO_SETUP, queryResult.getException(), queryResult.getWallTime(), queryResult.getCpuTime(), queryResult.getQueryId(), ImmutableList.>of()); } } @@ -294,7 +294,7 @@ private boolean checkForDeterministicAndRerunTestQueriesIfNeeded() private QueryResult executeQueryTest() { Query query = queryPair.getTest(); - QueryResult queryResult = new QueryResult(State.INVALID, null, null, null, null, ImmutableList.of()); + QueryResult queryResult = new QueryResult(State.INVALID, null, null, null, null, ImmutableList.>of()); try { // startup queryResult = setup(query, testPreQueryResults, testPrequery -> executeQuery(testGateway, testUsername, testPassword, queryPair.getTest(), testPrequery, testTimeout, sessionProperties)); @@ -331,7 +331,7 @@ private QueryResult executeQueryTest() private QueryResult executeQueryControl() { Query query = queryPair.getControl(); - QueryResult queryResult = new QueryResult(State.INVALID, null, null, null, null, ImmutableList.of()); + QueryResult queryResult = new QueryResult(State.INVALID, null, null, null, null, ImmutableList.>of()); try { // startup queryResult = setup(query, controlPreQueryResults, controlPrequery -> executeQuery(controlGateway, controlUsername, controlPassword, queryPair.getControl(), controlPrequery, controlTimeout, sessionProperties)); @@ -443,7 +443,7 @@ private QueryResult executeQuery(String url, String username, String password, Q } catch (AssertionError e) { if (e.getMessage().startsWith("unimplemented type:")) { - return new QueryResult(State.INVALID, null, null, null, queryId, ImmutableList.of()); + return new QueryResult(State.INVALID, null, null, null, queryId, ImmutableList.>of()); } throw e; } @@ -451,7 +451,7 @@ private QueryResult executeQuery(String url, String username, String password, Q throw e; } catch (UncheckedTimeoutException e) { - return new QueryResult(State.TIMEOUT, null, null, null, queryId, ImmutableList.of()); + return new QueryResult(State.TIMEOUT, null, null, null, queryId, ImmutableList.>of()); } catch (InterruptedException e) { Thread.currentThread().interrupt(); From 2e28bcbd0330a9a6b1c82a8a7286f2b5f439f025 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 20 Dec 2016 11:20:15 -0800 Subject: [PATCH 143/331] invoke isReachable to test connection --- .../hive/util/PooledTTransportFactory.java | 33 +++++++++++++++---- 1 file changed, 26 insertions(+), 7 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java index 02ac8955c2cd..68efa43131cb 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java @@ -55,10 +55,23 @@ public PooledTTransportFactory(TTransportPool pool, String host, int port, this.metastoreAuthentication = requireNonNull(metastoreAuthentication, "metastoreAuthentication is null"); } + @Override + public void activateObject(PooledObject pooledObject) + throws Exception + { + pooledObject.getObject().flush(); + } + @Override public boolean validateObject(PooledObject pooledObject) { - return pooledObject.getObject().isOpen(); + try { + return (pooledObject.getObject().isOpen() && + ((PooledTTransport) pooledObject.getObject()).isReachable(timeoutMillis)); + } + catch (Exception e) { + return false; + } } @Override @@ -147,6 +160,12 @@ public TTransport getTTransport() return transport; } + public boolean isReachable(int timeoutMillis) + throws ClassCastException, IOException + { + return ((TSocket) transport).getSocket().getInetAddress().isReachable(timeoutMillis); + } + @Override public void close() { @@ -196,42 +215,42 @@ public void consumeBuffer(int len) @Override public void open() - throws TTransportException + throws TTransportException { transport.open(); } @Override public int readAll(byte[] bytes, int off, int len) - throws TTransportException + throws TTransportException { return transport.readAll(bytes, off, len); } @Override public int read(byte[] bytes, int off, int len) - throws TTransportException + throws TTransportException { return transport.read(bytes, off, len); } @Override public void write(byte[] bytes) - throws TTransportException + throws TTransportException { transport.write(bytes); } @Override public void write(byte[] bytes, int off, int len) - throws TTransportException + throws TTransportException { transport.write(bytes, off, len); } @Override public void flush() - throws TTransportException + throws TTransportException { transport.flush(); } From 33761228e3da561f86b6a4982237fb134b02f32a Mon Sep 17 00:00:00 2001 From: Thomas Sun Date: Wed, 1 Feb 2017 11:41:18 -0800 Subject: [PATCH 144/331] warn instead of fail when JVM is non-Oracle --- .../com/facebook/presto/server/PrestoSystemRequirements.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-main/src/main/java/com/facebook/presto/server/PrestoSystemRequirements.java b/presto-main/src/main/java/com/facebook/presto/server/PrestoSystemRequirements.java index fdf7bc5f1031..de32c7d23571 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/PrestoSystemRequirements.java +++ b/presto-main/src/main/java/com/facebook/presto/server/PrestoSystemRequirements.java @@ -42,7 +42,7 @@ public static void verifyJvmRequirements() { String vendor = StandardSystemProperty.JAVA_VENDOR.value(); if (!"Oracle Corporation".equals(vendor)) { - failRequirement("Presto requires an Oracle or OpenJDK JVM (found %s)", vendor); + warnRequirement("Presto requires an Oracle or OpenJDK JVM (found %s)", vendor); } //verifyJavaVersion(); From 54e8b8379877a96ee2bd1b21e7d6370fd9d2c718 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 2 Feb 2017 16:43:04 -0800 Subject: [PATCH 145/331] update version number 0.157-tw-0.30 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 46 files changed, 48 insertions(+), 48 deletions(-) diff --git a/pom.xml b/pom.xml index 4544307a43fa..a280ea953f98 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.157-tw-0.29 + 0.157-tw-0.30 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 25b2dfa6873b..35df40abda52 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 9464d398187b..45a002821674 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 351ea0688410..08a3bc918dca 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index d13a25aad0bd..6c1db26f736a 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 45160e8845c6..cd672da526a1 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 4ca2d25d5552..7168f7b7e840 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157-tw-0.29 + 0.157-tw-0.30 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 4660227dace9..a468d5a5fadb 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 92241b60b780..7792df5f6c46 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 70ce2e31fb2c..2fb94af0f48c 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 784141533e83..15c1f92d734a 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 96d2efbf15f1..e2859313b7ae 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 33c7c59d1464..034c46d9c9d1 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 9d91549fea3f..7f1f7b053fc6 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 2daad579d330..86eecffbd11c 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index d911b3d72990..d41c3272f403 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 17f770566608..029ee98f88ca 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index c0fe52ce0229..b5a820ba7d23 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index fb87edef3156..6ac3a2e505bd 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index d1cb20840cec..0d02a6cf3067 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 8514c25365b5..cdeff26cd1ac 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 8781a46e9ce7..25ef04c360f8 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index c0fbc160d1c2..a95559d0eee5 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 97ddd14fac53..0dea7a6667e4 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 367ab2fc95d8..8066c3dda74d 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index ff06402a5013..49fb31dc2353 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 8519ca166be6..7149cbfe5beb 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index a552f39fabbe..76add5f61dc8 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 578271f897bc..5224f557aaa2 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index b54d6eb8a7b6..6f6014ce8843 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 646e8ebeb0a9..62b36a6b025a 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 6095c86862f3..11f0ce38c7fa 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157-tw-0.29 + 0.157-tw-0.30 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 5bd2ec84bd73..34a4161f5c35 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 62aba5012cb2..a6ff0f1e8d0e 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index aa4683471ce7..71388b19af99 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index cb0adca83ada..4eab1cea13bd 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 4f975bd1e7e3..680434d83b01 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 48c5116da66e..44c9161de67f 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 1b0743a1c432..ca66bfcdcfbd 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 59eb0ee34a0b..0fdc644844c1 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index cca9f667d352..05fd49ec1af8 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 0d5b8b0e0a69..0f3ffb4e0d94 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 1dd7f0988428..97a15d30c69b 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.157-tw-0.29 + 0.157-tw-0.30 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 38b538532789..586637d074f8 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index c6733e132dbe..09e3a2b58e73 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index b06141de197f..94fac335050b 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.157-tw-0.29 + 0.157-tw-0.30 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.157-tw-0.29 + 0.157-tw-0.30 provided From be4517a78cae4da2a1c8a6dbbfc8051875e37bc1 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 1 Mar 2017 14:03:44 -0800 Subject: [PATCH 146/331] Release 0.166-tw-0.31 From d5269642c8eee9a03686d3564ab35be42f464149 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Mon, 13 Mar 2017 18:18:59 -0700 Subject: [PATCH 147/331] Initial commit to support LZO Thrift --- presto-hive/pom.xml | 24 + .../presto/hive/HiveClientModule.java | 1 + .../com/facebook/presto/hive/HiveUtil.java | 4 + .../presto/hive/ThriftHiveRecordCursor.java | 710 ++++++++++++++++++ .../hive/ThriftHiveRecordCursorProvider.java | 124 +++ .../mapred/input/HiveMultiInputFormat.java | 73 ++ .../elephantbird/serde/ThriftSerDe.java | 82 ++ .../twitter/hive/thrift/DummyClass.java | 190 +++++ 8 files changed, 1208 insertions(+) create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveRecordCursor.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveRecordCursorProvider.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/twitter/elephantbird/mapred/input/HiveMultiInputFormat.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/twitter/elephantbird/serde/ThriftSerDe.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/DummyClass.java diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index f6ae9a5b7d52..e76118853d35 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -226,6 +226,30 @@ jackson-databind + + com.twitter.elephantbird + elephant-bird-core + 4.14 + thrift9 + + + commons-logging + commons-logging + + + com.hadoop.gplcompression + hadoop-lzo + + + + + + commons-io + commons-io + 2.5 + runtime + + com.facebook.presto 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 6ed0f01b551a..991700734209 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 @@ -93,6 +93,7 @@ public void configure(Binder binder) recordCursorProviderBinder.addBinding().to(ParquetRecordCursorProvider.class).in(Scopes.SINGLETON); recordCursorProviderBinder.addBinding().to(ColumnarTextHiveRecordCursorProvider.class).in(Scopes.SINGLETON); recordCursorProviderBinder.addBinding().to(ColumnarBinaryHiveRecordCursorProvider.class).in(Scopes.SINGLETON); + recordCursorProviderBinder.addBinding().to(ThriftHiveRecordCursorProvider.class).in(Scopes.SINGLETON); recordCursorProviderBinder.addBinding().to(GenericHiveRecordCursorProvider.class).in(Scopes.SINGLETON); binder.bind(HivePartitionManager.class).in(Scopes.SINGLETON); 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 667512f1b9ad..c81f7a22a8c3 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 @@ -257,6 +257,10 @@ static boolean isSplittable(InputFormat inputFormat, FileSystem fileSystem return true; } + if (inputFormat.getClass().getSimpleName().equals("HiveMultiInputFormat")) { + return false; + } + // use reflection to get isSplittable method on FileInputFormat Method method = null; for (Class clazz = inputFormat.getClass(); clazz != null; clazz = clazz.getSuperclass()) { diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveRecordCursor.java new file mode 100644 index 000000000000..a29e74c9ec46 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveRecordCursor.java @@ -0,0 +1,710 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.hive; + +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.RecordCursor; +import com.facebook.presto.spi.block.Block; +import com.facebook.presto.spi.block.BlockBuilder; +import com.facebook.presto.spi.block.BlockBuilderStatus; +import com.facebook.presto.spi.block.InterleavedBlockBuilder; +import com.facebook.presto.spi.type.DecimalType; +import com.facebook.presto.spi.type.Decimals; +import com.facebook.presto.spi.type.Type; +import com.facebook.presto.spi.type.TypeManager; +import com.facebook.presto.twitter.hive.thrift.DummyClass; +import com.google.common.base.Throwables; +import io.airlift.log.Logger; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import org.apache.hadoop.hive.common.type.HiveChar; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.common.type.HiveVarchar; +import org.apache.hadoop.hive.serde2.Deserializer; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.RecordReader; +import org.joda.time.DateTimeZone; + +import java.io.IOException; +import java.math.BigInteger; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.TimeUnit; + +import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; +import static com.facebook.presto.hive.HiveErrorCode.HIVE_CURSOR_ERROR; +import static com.facebook.presto.hive.HiveUtil.closeWithSuppression; +import static com.facebook.presto.hive.HiveUtil.getDeserializer; +import static com.facebook.presto.hive.HiveUtil.isArrayType; +import static com.facebook.presto.hive.HiveUtil.isMapType; +import static com.facebook.presto.hive.HiveUtil.isRowType; +import static com.facebook.presto.hive.HiveUtil.isStructuralType; +import static com.facebook.presto.hive.util.Types.checkType; +import static com.facebook.presto.spi.type.BigintType.BIGINT; +import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; +import static com.facebook.presto.spi.type.Chars.isCharType; +import static com.facebook.presto.spi.type.Chars.trimSpacesAndTruncateToLength; +import static com.facebook.presto.spi.type.DateType.DATE; +import static com.facebook.presto.spi.type.Decimals.rescale; +import static com.facebook.presto.spi.type.DoubleType.DOUBLE; +import static com.facebook.presto.spi.type.IntegerType.INTEGER; +import static com.facebook.presto.spi.type.RealType.REAL; +import static com.facebook.presto.spi.type.SmallintType.SMALLINT; +import static com.facebook.presto.spi.type.TimestampType.TIMESTAMP; +import static com.facebook.presto.spi.type.TinyintType.TINYINT; +import static com.facebook.presto.spi.type.VarbinaryType.VARBINARY; +import static com.facebook.presto.spi.type.Varchars.isVarcharType; +import static com.facebook.presto.spi.type.Varchars.truncateToLength; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; +import static java.lang.Float.floatToRawIntBits; +import static java.lang.Math.max; +import static java.lang.Math.min; +import static java.util.Objects.requireNonNull; + +class ThriftHiveRecordCursor + implements RecordCursor +{ + private static final Logger log = Logger.get(ThriftHiveRecordCursor.class); + private final RecordReader recordReader; + private final K key; + private final V value; + + @SuppressWarnings("deprecation") + private final Deserializer deserializer; + + private final Type[] types; + private final HiveType[] hiveTypes; + private final int[] hiveIndexs; + + private final boolean[] loaded; + private final boolean[] booleans; + private final long[] longs; + private final double[] doubles; + private final Slice[] slices; + private final Object[] objects; + private final boolean[] nulls; + + private final long totalBytes; + private final DateTimeZone hiveStorageTimeZone; + + private long completedBytes; + private Object rowData; + private boolean closed; + + public ThriftHiveRecordCursor( + RecordReader recordReader, + long totalBytes, + Properties splitSchema, + List columns, + DateTimeZone hiveStorageTimeZone, + TypeManager typeManager) + { + requireNonNull(recordReader, "recordReader is null"); + checkArgument(totalBytes >= 0, "totalBytes is negative"); + requireNonNull(splitSchema, "splitSchema is null"); + requireNonNull(columns, "columns is null"); + requireNonNull(hiveStorageTimeZone, "hiveStorageTimeZone is null"); + + this.recordReader = recordReader; + this.totalBytes = totalBytes; + this.key = recordReader.createKey(); + this.value = recordReader.createValue(); + this.hiveStorageTimeZone = hiveStorageTimeZone; + + this.deserializer = getDeserializer(splitSchema); + + int size = columns.size(); + + this.types = new Type[size]; + this.hiveTypes = new HiveType[size]; + this.hiveIndexs = new int[size]; + + this.loaded = new boolean[size]; + this.booleans = new boolean[size]; + this.longs = new long[size]; + this.doubles = new double[size]; + this.slices = new Slice[size]; + this.objects = new Object[size]; + this.nulls = new boolean[size]; + + // initialize data columns + for (int i = 0; i < columns.size(); i++) { + HiveColumnHandle column = columns.get(i); + checkState(column.getColumnType() == REGULAR, "column type must be regular"); + + types[i] = typeManager.getType(column.getTypeSignature()); + hiveTypes[i] = column.getHiveType(); + hiveIndexs[i] = column.getHiveColumnIndex(); + } + } + + @Override + public long getTotalBytes() + { + return totalBytes; + } + + @Override + public long getCompletedBytes() + { + if (!closed) { + updateCompletedBytes(); + } + return completedBytes; + } + + @Override + public long getReadTimeNanos() + { + return 0; + } + + private void updateCompletedBytes() + { + try { + long newCompletedBytes = (long) (totalBytes * recordReader.getProgress()); + completedBytes = min(totalBytes, max(completedBytes, newCompletedBytes)); + } + catch (IOException ignored) { + } + } + + @Override + public Type getType(int field) + { + return types[field]; + } + + @Override + public boolean advanceNextPosition() + { + try { + if (closed || !recordReader.next(key, value)) { + close(); + return false; + } + + // reset loaded flags + Arrays.fill(loaded, false); + + // decode value + rowData = deserializer.deserialize(value); + + return true; + } + catch (IOException | SerDeException | RuntimeException e) { + closeWithSuppression(this, e); + throw new PrestoException(HIVE_CURSOR_ERROR, e); + } + } + + @Override + public boolean getBoolean(int fieldId) + { + checkState(!closed, "Cursor is closed"); + + validateType(fieldId, boolean.class); + if (!loaded[fieldId]) { + parseBooleanColumn(fieldId); + } + return booleans[fieldId]; + } + + private void parseBooleanColumn(int column) + { + loaded[column] = true; + + DummyClass dummy = (DummyClass) rowData; + Object fieldValue = dummy.values.get((short) hiveIndexs[column]); + + if (fieldValue == null) { + nulls[column] = true; + } + else { + booleans[column] = (Boolean) fieldValue; + nulls[column] = false; + } + } + + @Override + public long getLong(int fieldId) + { + checkState(!closed, "Cursor is closed"); + + validateType(fieldId, long.class); + if (!loaded[fieldId]) { + parseLongColumn(fieldId); + } + return longs[fieldId]; + } + + private void parseLongColumn(int column) + { + loaded[column] = true; + + DummyClass dummy = (DummyClass) rowData; + Object fieldValue = dummy.values.get((short) hiveIndexs[column]); + + if (fieldValue == null) { + nulls[column] = true; + } + else { + longs[column] = getLongExpressedValue(fieldValue, hiveStorageTimeZone); + nulls[column] = false; + } + } + + private static long getLongExpressedValue(Object value, DateTimeZone hiveTimeZone) + { + if (value instanceof Date) { + long storageTime = ((Date) value).getTime(); + // convert date from VM current time zone to UTC + long utcMillis = storageTime + DateTimeZone.getDefault().getOffset(storageTime); + return TimeUnit.MILLISECONDS.toDays(utcMillis); + } + if (value instanceof Timestamp) { + // The Hive SerDe parses timestamps using the default time zone of + // this JVM, but the data might have been written using a different + // time zone. We need to convert it to the configured time zone. + + // the timestamp that Hive parsed using the JVM time zone + long parsedJvmMillis = ((Timestamp) value).getTime(); + + // remove the JVM time zone correction from the timestamp + DateTimeZone jvmTimeZone = DateTimeZone.getDefault(); + long hiveMillis = jvmTimeZone.convertUTCToLocal(parsedJvmMillis); + + // convert to UTC using the real time zone for the underlying data + long utcMillis = hiveTimeZone.convertLocalToUTC(hiveMillis, false); + + return utcMillis; + } + if (value instanceof Float) { + return floatToRawIntBits(((Float) value)); + } + return ((Number) value).longValue(); + } + + @Override + public double getDouble(int fieldId) + { + checkState(!closed, "Cursor is closed"); + + validateType(fieldId, double.class); + if (!loaded[fieldId]) { + parseDoubleColumn(fieldId); + } + return doubles[fieldId]; + } + + private void parseDoubleColumn(int column) + { + loaded[column] = true; + + DummyClass dummy = (DummyClass) rowData; + Object fieldValue = dummy.values.get((short) hiveIndexs[column]); + + if (fieldValue == null) { + nulls[column] = true; + } + else { + doubles[column] = ((Number) fieldValue).doubleValue(); + nulls[column] = false; + } + } + + @Override + public Slice getSlice(int fieldId) + { + checkState(!closed, "Cursor is closed"); + + validateType(fieldId, Slice.class); + if (!loaded[fieldId]) { + parseStringColumn(fieldId); + } + return slices[fieldId]; + } + + private void parseStringColumn(int column) + { + loaded[column] = true; + + DummyClass dummy = (DummyClass) rowData; + Object fieldValue = dummy.values.get((short) hiveIndexs[column]); + + if (fieldValue == null) { + nulls[column] = true; + } + else { + slices[column] = getSliceExpressedValue(fieldValue, types[column]); + nulls[column] = false; + } + } + + private static Slice getSliceExpressedValue(Object value, Type type) + { + Slice sliceValue; + if (value instanceof String) { + sliceValue = Slices.utf8Slice((String) value); + } + else if (value instanceof byte[]) { + sliceValue = Slices.wrappedBuffer((byte[]) value); + } + else if (value instanceof HiveVarchar) { + sliceValue = Slices.utf8Slice(((HiveVarchar) value).getValue()); + } + else if (value instanceof HiveChar) { + sliceValue = Slices.utf8Slice(((HiveChar) value).getValue()); + } + else if (value instanceof Integer) { + sliceValue = Slices.utf8Slice(value.toString()); + } + else { + throw new IllegalStateException("unsupported string field type: " + value.getClass().getName()); + } + if (isVarcharType(type)) { + sliceValue = truncateToLength(sliceValue, type); + } + if (isCharType(type)) { + sliceValue = trimSpacesAndTruncateToLength(sliceValue, type); + } + + return sliceValue; + } + + private void parseDecimalColumn(int column) + { + loaded[column] = true; + + DummyClass dummy = (DummyClass) rowData; + Object fieldValue = dummy.values.get((short) hiveIndexs[column]); + + if (fieldValue == null) { + nulls[column] = true; + } + else { + HiveDecimal decimal = (HiveDecimal) fieldValue; + DecimalType columnType = (DecimalType) types[column]; + BigInteger unscaledDecimal = rescale(decimal.unscaledValue(), decimal.scale(), columnType.getScale()); + + if (columnType.isShort()) { + longs[column] = unscaledDecimal.longValue(); + } + else { + slices[column] = Decimals.encodeUnscaledValue(unscaledDecimal); + } + nulls[column] = false; + } + } + + @Override + public Object getObject(int fieldId) + { + checkState(!closed, "Cursor is closed"); + + validateType(fieldId, Block.class); + if (!loaded[fieldId]) { + parseObjectColumn(fieldId); + } + return objects[fieldId]; + } + + private void parseObjectColumn(int column) + { + loaded[column] = true; + + DummyClass dummy = (DummyClass) rowData; + Object fieldValue = dummy.values.get((short) hiveIndexs[column]); + + if (fieldValue == null) { + nulls[column] = true; + } + else { + objects[column] = getBlockObject(types[column], fieldValue, hiveStorageTimeZone); + nulls[column] = false; + } + } + + @Override + public boolean isNull(int fieldId) + { + checkState(!closed, "Cursor is closed"); + + if (!loaded[fieldId]) { + parseColumn(fieldId); + } + return nulls[fieldId]; + } + + private void parseColumn(int column) + { + Type type = types[column]; + if (BOOLEAN.equals(type)) { + parseBooleanColumn(column); + } + else if (BIGINT.equals(type)) { + parseLongColumn(column); + } + else if (INTEGER.equals(type)) { + parseLongColumn(column); + } + else if (SMALLINT.equals(type)) { + parseLongColumn(column); + } + else if (TINYINT.equals(type)) { + parseLongColumn(column); + } + else if (REAL.equals(type)) { + parseLongColumn(column); + } + else if (DOUBLE.equals(type)) { + parseDoubleColumn(column); + } + else if (isVarcharType(type) || VARBINARY.equals(type)) { + parseStringColumn(column); + } + else if (isCharType(type)) { + parseStringColumn(column); + } + else if (isStructuralType(hiveTypes[column])) { + parseObjectColumn(column); + } + else if (DATE.equals(type)) { + parseLongColumn(column); + } + else if (TIMESTAMP.equals(type)) { + parseLongColumn(column); + } + else if (type instanceof DecimalType) { + parseDecimalColumn(column); + } + else { + throw new UnsupportedOperationException("Unsupported column type: " + type); + } + } + + private void validateType(int fieldId, Class type) + { + if (!types[fieldId].getJavaType().equals(type)) { + // we don't use Preconditions.checkArgument because it requires boxing fieldId, which affects inner loop performance + throw new IllegalArgumentException(String.format("Expected field to be %s, actual %s (field %s)", type, types[fieldId], fieldId)); + } + } + + @Override + public void close() + { + // some hive input formats are broken and bad things can happen if you close them multiple times + if (closed) { + return; + } + closed = true; + + updateCompletedBytes(); + + try { + recordReader.close(); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + + private static Block getBlockObject(Type type, Object object, DateTimeZone hiveStorageTimeZone) + { + return requireNonNull(serializeObject(type, null, object, hiveStorageTimeZone), "serialized result is null"); + } + + private static Block serializeObject(Type type, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) + { + if (!isStructuralType(type)) { + serializePrimitive(type, builder, object, hiveStorageTimeZone); + return null; + } + else if (isArrayType(type)) { + return serializeList(type, builder, object, hiveStorageTimeZone); + } + else if (isMapType(type)) { + return serializeMap(type, builder, object, hiveStorageTimeZone); + } + else if (isRowType(type)) { + return serializeStruct(type, builder, object, hiveStorageTimeZone); + } + throw new RuntimeException("Unknown object type: " + type); + } + + private static Block serializeList(Type type, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) + { + List list = (List) object; + if (list == null) { + requireNonNull(builder, "parent builder is null").appendNull(); + return null; + } + + List typeParameters = type.getTypeParameters(); + checkArgument(typeParameters.size() == 1, "list must have exactly 1 type parameter"); + Type elementType = typeParameters.get(0); + BlockBuilder currentBuilder; + if (builder != null) { + currentBuilder = builder.beginBlockEntry(); + } + else { + currentBuilder = elementType.createBlockBuilder(new BlockBuilderStatus(), list.size()); + } + + for (Object element : list) { + serializeObject(elementType, currentBuilder, element, hiveStorageTimeZone); + } + + if (builder != null) { + builder.closeEntry(); + return null; + } + else { + Block resultBlock = currentBuilder.build(); + return resultBlock; + } + } + + private static Block serializeMap(Type type, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) + { + Map map = (Map) object; + if (map == null) { + requireNonNull(builder, "parent builder is null").appendNull(); + return null; + } + + List typeParameters = type.getTypeParameters(); + checkArgument(typeParameters.size() == 2, "map must have exactly 2 type parameter"); + Type keyType = typeParameters.get(0); + Type valueType = typeParameters.get(1); + BlockBuilder currentBuilder; + if (builder != null) { + currentBuilder = builder.beginBlockEntry(); + } + else { + currentBuilder = new InterleavedBlockBuilder(typeParameters, new BlockBuilderStatus(), map.size()); + } + + for (Map.Entry entry : map.entrySet()) { + // Hive skips map entries with null keys + if (entry.getKey() != null) { + serializeObject(keyType, currentBuilder, entry.getKey(), hiveStorageTimeZone); + serializeObject(valueType, currentBuilder, entry.getValue(), hiveStorageTimeZone); + } + } + + if (builder != null) { + builder.closeEntry(); + return null; + } + else { + Block resultBlock = currentBuilder.build(); + return resultBlock; + } + } + + private static Block serializeStruct(Type type, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) + { + if (object == null) { + requireNonNull(builder, "parent builder is null").appendNull(); + return null; + } + + List typeParameters = type.getTypeParameters(); + Map allValues = ((DummyClass) object).values; + BlockBuilder currentBuilder; + if (builder != null) { + currentBuilder = builder.beginBlockEntry(); + } + else { + currentBuilder = new InterleavedBlockBuilder(typeParameters, new BlockBuilderStatus(), typeParameters.size()); + } + + for (int i = 0; i < typeParameters.size(); i++) { + Object field = allValues.get((short) i); + serializeObject(typeParameters.get(i), currentBuilder, field, hiveStorageTimeZone); + } + + if (builder != null) { + builder.closeEntry(); + return null; + } + else { + Block resultBlock = currentBuilder.build(); + return resultBlock; + } + } + + private static void serializePrimitive(Type type, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) + { + requireNonNull(builder, "parent builder is null"); + + if (object == null) { + builder.appendNull(); + return; + } + + if (BOOLEAN.equals(type)) { + BOOLEAN.writeBoolean(builder, (Boolean) object); + } + else if (BIGINT.equals(type)) { + BIGINT.writeLong(builder, getLongExpressedValue(object, hiveStorageTimeZone)); + } + else if (INTEGER.equals(type)) { + INTEGER.writeLong(builder, getLongExpressedValue(object, hiveStorageTimeZone)); + } + else if (SMALLINT.equals(type)) { + SMALLINT.writeLong(builder, getLongExpressedValue(object, hiveStorageTimeZone)); + } + else if (TINYINT.equals(type)) { + TINYINT.writeLong(builder, getLongExpressedValue(object, hiveStorageTimeZone)); + } + else if (REAL.equals(type)) { + REAL.writeLong(builder, getLongExpressedValue(object, hiveStorageTimeZone)); + } + else if (DOUBLE.equals(type)) { + DOUBLE.writeDouble(builder, ((Number) object).doubleValue()); + } + else if (isVarcharType(type) || VARBINARY.equals(type) || isCharType(type)) { + type.writeSlice(builder, getSliceExpressedValue(object, type)); + } + else if (DATE.equals(type)) { + DATE.writeLong(builder, getLongExpressedValue(object, hiveStorageTimeZone)); + } + else if (TIMESTAMP.equals(type)) { + TIMESTAMP.writeLong(builder, getLongExpressedValue(object, hiveStorageTimeZone)); + } + else if (type instanceof DecimalType) { + HiveDecimal decimal = (HiveDecimal) object; + DecimalType decimalType = checkType(type, DecimalType.class, "type"); + BigInteger unscaledDecimal = rescale(decimal.unscaledValue(), decimal.scale(), decimalType.getScale()); + if (decimalType.isShort()) { + decimalType.writeLong(builder, unscaledDecimal.longValue()); + } + else { + decimalType.writeSlice(builder, Decimals.encodeUnscaledValue(unscaledDecimal)); + } + } + else { + throw new UnsupportedOperationException("Unsupported primitive type: " + type); + } + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveRecordCursorProvider.java b/presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveRecordCursorProvider.java new file mode 100644 index 000000000000..d3a3edd7bdd6 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveRecordCursorProvider.java @@ -0,0 +1,124 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.hive; + +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.RecordCursor; +import com.facebook.presto.spi.predicate.TupleDomain; +import com.facebook.presto.spi.type.TypeManager; +import com.google.common.collect.ImmutableSet; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.RecordReader; +import org.joda.time.DateTimeZone; + +import javax.inject.Inject; + +import java.io.IOException; +import java.util.List; +import java.util.Optional; +import java.util.Properties; +import java.util.Set; + +import static com.facebook.presto.hive.HiveUtil.getDeserializerClassName; +import static java.util.Objects.requireNonNull; + +public class ThriftHiveRecordCursorProvider + implements HiveRecordCursorProvider +{ + private static final Set THRIFT_SERDE_CLASS_NAMES = ImmutableSet.builder() + .add("com.twitter.elephantbird.hive.serde.ThriftSerDe") + .build(); + private final HdfsEnvironment hdfsEnvironment; + + @Inject + public ThriftHiveRecordCursorProvider(HdfsEnvironment hdfsEnvironment) + { + this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); + } + + @Override + public Optional createRecordCursor( + String clientId, + Configuration configuration, + ConnectorSession session, + Path path, + long start, + long length, + Properties schema, + List columns, + TupleDomain effectivePredicate, + DateTimeZone hiveStorageTimeZone, + TypeManager typeManager) + { + if (!THRIFT_SERDE_CLASS_NAMES.contains(getDeserializerClassName(schema))) { + return Optional.empty(); + } + + RecordReader recordReader; + if (path.toString().endsWith(".index")) { + recordReader = new DummyRecordReader(); + } + else { + recordReader = hdfsEnvironment.doAs(session.getUser(), + () -> HiveUtil.createRecordReader(configuration, path, start, length, schema, columns)); + } + + return Optional.of(new ThriftHiveRecordCursor<>( + genericRecordReader(recordReader), + length, + schema, + columns, + hiveStorageTimeZone, + typeManager)); + } + + @SuppressWarnings("unchecked") + private static RecordReader genericRecordReader(RecordReader recordReader) + { + return (RecordReader) recordReader; + } + + private static final class DummyRecordReader implements RecordReader + { + public boolean next(K key, V value) throws IOException + { + return false; + } + + public K createKey() + { + return null; + } + + public V createValue() + { + return null; + } + + public long getPos() throws IOException + { + return 0; + } + + public float getProgress() throws IOException + { + return 0; + } + + public void close() throws IOException {} + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/elephantbird/mapred/input/HiveMultiInputFormat.java b/presto-hive/src/main/java/com/facebook/presto/twitter/elephantbird/mapred/input/HiveMultiInputFormat.java new file mode 100644 index 000000000000..ab457a37a4b5 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/elephantbird/mapred/input/HiveMultiInputFormat.java @@ -0,0 +1,73 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.twitter.elephantbird.mapred.input; + +import com.twitter.elephantbird.mapred.input.DeprecatedFileInputFormatWrapper; +import com.twitter.elephantbird.mapreduce.input.MultiInputFormat; +import com.twitter.elephantbird.mapreduce.io.BinaryWritable; +import com.twitter.elephantbird.util.TypeRef; +import io.airlift.log.Logger; +import org.apache.hadoop.hive.serde.Constants; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.mapred.FileSplit; +import org.apache.hadoop.mapred.InputSplit; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.RecordReader; +import org.apache.hadoop.mapred.Reporter; + +import java.io.IOException; + +/** + * Customized version of com.twitter.elephantbird.mapred.input.HiveMultiInputFormat + */ +@SuppressWarnings("deprecation") +public class HiveMultiInputFormat extends DeprecatedFileInputFormatWrapper +{ + private static final Logger log = Logger.get(HiveMultiInputFormat.class); + + public HiveMultiInputFormat() + { + super(new MultiInputFormat()); + } + + private void initialize(FileSplit split, JobConf job) throws IOException + { + log.info("Initializing HiveMultiInputFormat for " + split + " with job " + job); + + String thriftClassName = null; + + thriftClassName = job.get(Constants.SERIALIZATION_CLASS); + + if (thriftClassName == null) { + throw new RuntimeException( + "Required property " + Constants.SERIALIZATION_CLASS + " is null."); + } + + try { + Class thriftClass = job.getClassByName(thriftClassName); + setInputFormatInstance(new MultiInputFormat(new TypeRef(thriftClass) {})); + } + catch (ClassNotFoundException e) { + throw new RuntimeException("Failed getting class for " + thriftClassName); + } + } + + @Override + public RecordReader getRecordReader(InputSplit split, JobConf job, + Reporter reporter) throws IOException + { + initialize((FileSplit) split, job); + return super.getRecordReader(split, job, reporter); + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/elephantbird/serde/ThriftSerDe.java b/presto-hive/src/main/java/com/facebook/presto/twitter/elephantbird/serde/ThriftSerDe.java new file mode 100644 index 000000000000..2c312a33c559 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/elephantbird/serde/ThriftSerDe.java @@ -0,0 +1,82 @@ +/* + * 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.twitter.elephantbird.hive.serde; + +import com.twitter.elephantbird.mapreduce.io.ThriftWritable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.serde.Constants; +import org.apache.hadoop.hive.serde2.SerDe; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.SerDeStats; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.io.Writable; + +import java.util.Properties; + +/** + * SerDe for working with {@link ThriftWritable} records. + * This pairs well with {@link com.twitter.elephantbird.mapred.input.HiveMultiInputFormat}. + */ +public class ThriftSerDe implements SerDe +{ + @Override + public void initialize(Configuration conf, Properties properties) throws SerDeException + { + String thriftClassName = properties.getProperty(Constants.SERIALIZATION_CLASS, null); + if (thriftClassName == null) { + throw new SerDeException("Required property " + Constants.SERIALIZATION_CLASS + " is null."); + } + + Class thriftClass; + try { + thriftClass = conf.getClassByName(thriftClassName); + } + catch (ClassNotFoundException e) { + throw new SerDeException("Failed getting class for " + thriftClassName); + } + } + + @Override + public Class getSerializedClass() + { + return null; + } + + @Override + public Writable serialize(Object o, ObjectInspector objectInspector) throws SerDeException + { + return null; + } + + @Override + public Object deserialize(Writable writable) throws SerDeException + { + if (!(writable instanceof ThriftWritable)) { + throw new SerDeException("Not an instance of ThriftWritable: " + writable); + } + return ((ThriftWritable) writable).get(); + } + + @Override + public ObjectInspector getObjectInspector() throws SerDeException + { + return null; + } + + @Override + public SerDeStats getSerDeStats() + { + return null; + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/DummyClass.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/DummyClass.java new file mode 100644 index 000000000000..1c76360a8b98 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/DummyClass.java @@ -0,0 +1,190 @@ +/* + * 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.twitter.hive.thrift; + +import io.airlift.log.Logger; +import org.apache.thrift.TBase; +import org.apache.thrift.TException; +import org.apache.thrift.TFieldIdEnum; +import org.apache.thrift.protocol.TField; +import org.apache.thrift.protocol.TList; +import org.apache.thrift.protocol.TMap; +import org.apache.thrift.protocol.TProtocol; +import org.apache.thrift.protocol.TProtocolUtil; +import org.apache.thrift.protocol.TSet; +import org.apache.thrift.protocol.TType; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class DummyClass implements TBase +{ + private static final Logger log = Logger.get(DummyClass.class); + public final Map values = new HashMap<>(); + + public DummyClass() {} + + public class Fields implements TFieldIdEnum + { + private final short thriftId; + private final String fieldName; + + Fields(short thriftId, String fieldName) + { + this.thriftId = thriftId; + this.fieldName = fieldName; + } + + public short getThriftFieldId() + { + return thriftId; + } + + public String getFieldName() + { + return fieldName; + } + } + + public DummyClass deepCopy() + { + return new DummyClass(); + } + + public void clear() {} + + public Fields fieldForId(int fieldId) + { + return new Fields((short) fieldId, "dummy"); + } + + public Object getFieldValue(Fields field) + { + return values.get(field.thriftId); + } + + public boolean isSet(Fields field) + { + return values.containsKey(field.getThriftFieldId()); + } + + public void read(TProtocol iprot) throws TException + { + TField field; + // int i = 0; + iprot.readStructBegin(); + while (true) { + field = iprot.readFieldBegin(); + if (field.type == TType.STOP) { + break; + } + // NOTES: The thrift id may not the same of hive table column number + // We can either fill the hive table with discontinued id or discard + // the field.id here and use a counter as its column number. Like: + // values.put((short) ++i, readElem(iprot, field.type)); + values.put((short) (field.id - 1), readElem(iprot, field.type)); + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + } + + private Object readElem(TProtocol iprot, byte type) throws TException + { + switch (type) { + case TType.BOOL: + return iprot.readBool(); + case TType.BYTE: + return iprot.readByte(); + case TType.I16: + return iprot.readI16(); + case TType.ENUM: + case TType.I32: + return iprot.readI32(); + case TType.I64: + return iprot.readI64(); + case TType.DOUBLE: + return iprot.readDouble(); + case TType.STRING: + return iprot.readString(); + case TType.STRUCT: + return readStruct(iprot); + case TType.LIST: + return readList(iprot); + case TType.SET: + return readSet(iprot); + case TType.MAP: + return readMap(iprot); + default: + TProtocolUtil.skip(iprot, type); + } + return null; + } + + private Object readStruct(TProtocol iprot) throws TException + { + DummyClass elem = new DummyClass(); + elem.read(iprot); + return elem; + } + + private Object readList(TProtocol iprot) throws TException + { + TList ilist = iprot.readListBegin(); + List listValue = new ArrayList<>(); + for (int i = 0; i < ilist.size; ++i) { + listValue.add(readElem(iprot, ilist.elemType)); + } + iprot.readListEnd(); + return listValue; + } + + private Object readSet(TProtocol iprot) throws TException + { + TSet iset = iprot.readSetBegin(); + List setValue = new ArrayList<>(); + for (int i = 0; i < iset.size; ++i) { + setValue.add(readElem(iprot, iset.elemType)); + } + iprot.readSetEnd(); + return setValue; + } + + private Object readMap(TProtocol iprot) throws TException + { + TMap imap = iprot.readMapBegin(); + Map mapValue = new HashMap<>(); + for (int i = 0; i < imap.size; ++i) { + mapValue.put(readElem(iprot, imap.keyType), readElem(iprot, imap.valueType)); + } + iprot.readMapEnd(); + return mapValue; + } + + public void setFieldValue(Fields field, Object value) + { + throw new UnsupportedOperationException("DummyClass.setFieldValue is not supported."); + } + + public void write(TProtocol oprot) + { + throw new UnsupportedOperationException("DummyClass.write is not supported."); + } + + public int compareTo(DummyClass other) + { + throw new UnsupportedOperationException("Dummy.compareTo is not supported."); + } +} From 9027039c1d6692971a8f522bc02218122346f15f Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 17 Mar 2017 17:41:38 -0700 Subject: [PATCH 148/331] Move classes into twitter folder with better namings --- .../presto/hive/HiveClientModule.java | 1 + .../com/facebook/presto/hive/HiveUtil.java | 2 +- .../elephantbird/serde/ThriftSerDe.java | 82 ------------------- .../thrift/ThriftGeneralDeserializer.java | 44 ++++++++++ .../thrift/ThriftGeneralInputFormat.java} | 26 ++---- ...{DummyClass.java => ThriftGeneralRow.java} | 79 +++++++++--------- .../hive/thrift}/ThriftHiveRecordCursor.java | 43 ++++------ .../ThriftHiveRecordCursorProvider.java | 11 ++- 8 files changed, 123 insertions(+), 165 deletions(-) delete mode 100644 presto-hive/src/main/java/com/facebook/presto/twitter/elephantbird/serde/ThriftSerDe.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralDeserializer.java rename presto-hive/src/main/java/com/facebook/presto/twitter/{elephantbird/mapred/input/HiveMultiInputFormat.java => hive/thrift/ThriftGeneralInputFormat.java} (72%) rename presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/{DummyClass.java => ThriftGeneralRow.java} (81%) rename presto-hive/src/main/java/com/facebook/presto/{hive => twitter/hive/thrift}/ThriftHiveRecordCursor.java (94%) rename presto-hive/src/main/java/com/facebook/presto/{hive => twitter/hive/thrift}/ThriftHiveRecordCursorProvider.java (87%) 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 991700734209..dd0406d0d47b 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 @@ -27,6 +27,7 @@ import com.facebook.presto.spi.connector.ConnectorSplitManager; import com.facebook.presto.spi.type.TypeManager; import com.facebook.presto.twitter.hive.PooledHiveMetastoreClientFactory; +import com.facebook.presto.twitter.hive.thrift.ThriftHiveRecordCursorProvider; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Provides; 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 c81f7a22a8c3..356101d3c5e6 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 @@ -257,7 +257,7 @@ static boolean isSplittable(InputFormat inputFormat, FileSystem fileSystem return true; } - if (inputFormat.getClass().getSimpleName().equals("HiveMultiInputFormat")) { + if (inputFormat.getClass().getSimpleName().equals("ThriftGeneralInputFormat")) { return false; } diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/elephantbird/serde/ThriftSerDe.java b/presto-hive/src/main/java/com/facebook/presto/twitter/elephantbird/serde/ThriftSerDe.java deleted file mode 100644 index 2c312a33c559..000000000000 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/elephantbird/serde/ThriftSerDe.java +++ /dev/null @@ -1,82 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.twitter.elephantbird.hive.serde; - -import com.twitter.elephantbird.mapreduce.io.ThriftWritable; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.serde.Constants; -import org.apache.hadoop.hive.serde2.SerDe; -import org.apache.hadoop.hive.serde2.SerDeException; -import org.apache.hadoop.hive.serde2.SerDeStats; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.io.Writable; - -import java.util.Properties; - -/** - * SerDe for working with {@link ThriftWritable} records. - * This pairs well with {@link com.twitter.elephantbird.mapred.input.HiveMultiInputFormat}. - */ -public class ThriftSerDe implements SerDe -{ - @Override - public void initialize(Configuration conf, Properties properties) throws SerDeException - { - String thriftClassName = properties.getProperty(Constants.SERIALIZATION_CLASS, null); - if (thriftClassName == null) { - throw new SerDeException("Required property " + Constants.SERIALIZATION_CLASS + " is null."); - } - - Class thriftClass; - try { - thriftClass = conf.getClassByName(thriftClassName); - } - catch (ClassNotFoundException e) { - throw new SerDeException("Failed getting class for " + thriftClassName); - } - } - - @Override - public Class getSerializedClass() - { - return null; - } - - @Override - public Writable serialize(Object o, ObjectInspector objectInspector) throws SerDeException - { - return null; - } - - @Override - public Object deserialize(Writable writable) throws SerDeException - { - if (!(writable instanceof ThriftWritable)) { - throw new SerDeException("Not an instance of ThriftWritable: " + writable); - } - return ((ThriftWritable) writable).get(); - } - - @Override - public ObjectInspector getObjectInspector() throws SerDeException - { - return null; - } - - @Override - public SerDeStats getSerDeStats() - { - return null; - } -} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralDeserializer.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralDeserializer.java new file mode 100644 index 000000000000..cddd4f1f85e5 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralDeserializer.java @@ -0,0 +1,44 @@ +/* + * 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.twitter.hive.thrift; + +import com.twitter.elephantbird.mapreduce.io.ThriftWritable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.Writable; + +import java.util.Properties; + +import static com.facebook.presto.hive.HiveErrorCode.HIVE_INVALID_METADATA; +import static com.facebook.presto.hive.HiveErrorCode.HIVE_UNKNOWN_ERROR; +import static com.facebook.presto.hive.HiveUtil.checkCondition; +import static org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS; + +public class ThriftGeneralDeserializer +{ + private static final String REQUIRED_SERIALIZATION_CLASS = "com.facebook.presto.twitter.hive.thrift.ThriftGeneralRow"; + + public void initialize(Configuration conf, Properties properties) + { + String thriftClassName = properties.getProperty(SERIALIZATION_CLASS, null); + checkCondition(thriftClassName != null, HIVE_INVALID_METADATA, "Table or partition is missing Hive deserializer property: %s", SERIALIZATION_CLASS); + checkCondition(thriftClassName.equals(REQUIRED_SERIALIZATION_CLASS), HIVE_INVALID_METADATA, SERIALIZATION_CLASS + thriftClassName + " cannot match " + REQUIRED_SERIALIZATION_CLASS); + return; + } + + public ThriftGeneralRow deserialize(Writable writable) + { + checkCondition(writable instanceof ThriftWritable, HIVE_UNKNOWN_ERROR, "Not an instance of ThriftWritable: " + writable); + return (ThriftGeneralRow) ((ThriftWritable) writable).get(); + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/elephantbird/mapred/input/HiveMultiInputFormat.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java similarity index 72% rename from presto-hive/src/main/java/com/facebook/presto/twitter/elephantbird/mapred/input/HiveMultiInputFormat.java rename to presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java index ab457a37a4b5..e92560729804 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/elephantbird/mapred/input/HiveMultiInputFormat.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java @@ -11,14 +11,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.facebook.presto.twitter.elephantbird.mapred.input; +package com.facebook.presto.twitter.hive.thrift; import com.twitter.elephantbird.mapred.input.DeprecatedFileInputFormatWrapper; import com.twitter.elephantbird.mapreduce.input.MultiInputFormat; import com.twitter.elephantbird.mapreduce.io.BinaryWritable; import com.twitter.elephantbird.util.TypeRef; -import io.airlift.log.Logger; -import org.apache.hadoop.hive.serde.Constants; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.mapred.InputSplit; @@ -28,31 +26,25 @@ import java.io.IOException; +import static com.facebook.presto.hive.HiveErrorCode.HIVE_INVALID_METADATA; +import static com.facebook.presto.hive.HiveUtil.checkCondition; +import static org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS; + /** * Customized version of com.twitter.elephantbird.mapred.input.HiveMultiInputFormat */ @SuppressWarnings("deprecation") -public class HiveMultiInputFormat extends DeprecatedFileInputFormatWrapper +public class ThriftGeneralInputFormat extends DeprecatedFileInputFormatWrapper { - private static final Logger log = Logger.get(HiveMultiInputFormat.class); - - public HiveMultiInputFormat() + public ThriftGeneralInputFormat() { super(new MultiInputFormat()); } private void initialize(FileSplit split, JobConf job) throws IOException { - log.info("Initializing HiveMultiInputFormat for " + split + " with job " + job); - - String thriftClassName = null; - - thriftClassName = job.get(Constants.SERIALIZATION_CLASS); - - if (thriftClassName == null) { - throw new RuntimeException( - "Required property " + Constants.SERIALIZATION_CLASS + " is null."); - } + String thriftClassName = job.get(SERIALIZATION_CLASS); + checkCondition(thriftClassName != null, HIVE_INVALID_METADATA, "Table or partition is missing Hive deserializer property: %s", SERIALIZATION_CLASS); try { Class thriftClass = job.getClassByName(thriftClassName); diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/DummyClass.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralRow.java similarity index 81% rename from presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/DummyClass.java rename to presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralRow.java index 1c76360a8b98..689298606606 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/DummyClass.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralRow.java @@ -30,12 +30,17 @@ import java.util.List; import java.util.Map; -public class DummyClass implements TBase +public class ThriftGeneralRow implements TBase { - private static final Logger log = Logger.get(DummyClass.class); - public final Map values = new HashMap<>(); + private static final Logger log = Logger.get(ThriftGeneralRow.class); + private final Map values = new HashMap<>(); - public DummyClass() {} + public ThriftGeneralRow() {} + + public ThriftGeneralRow(Map values) + { + this.values.putAll(values); + } public class Fields implements TFieldIdEnum { @@ -59,43 +64,16 @@ public String getFieldName() } } - public DummyClass deepCopy() - { - return new DummyClass(); - } - - public void clear() {} - - public Fields fieldForId(int fieldId) - { - return new Fields((short) fieldId, "dummy"); - } - - public Object getFieldValue(Fields field) - { - return values.get(field.thriftId); - } - - public boolean isSet(Fields field) - { - return values.containsKey(field.getThriftFieldId()); - } - public void read(TProtocol iprot) throws TException { TField field; - // int i = 0; iprot.readStructBegin(); while (true) { field = iprot.readFieldBegin(); if (field.type == TType.STOP) { break; } - // NOTES: The thrift id may not the same of hive table column number - // We can either fill the hive table with discontinued id or discard - // the field.id here and use a counter as its column number. Like: - // values.put((short) ++i, readElem(iprot, field.type)); - values.put((short) (field.id - 1), readElem(iprot, field.type)); + values.put(field.id, readElem(iprot, field.type)); iprot.readFieldEnd(); } iprot.readStructEnd(); @@ -135,7 +113,7 @@ private Object readElem(TProtocol iprot, byte type) throws TException private Object readStruct(TProtocol iprot) throws TException { - DummyClass elem = new DummyClass(); + ThriftGeneralRow elem = new ThriftGeneralRow(); elem.read(iprot); return elem; } @@ -173,18 +151,45 @@ private Object readMap(TProtocol iprot) throws TException return mapValue; } + public Object getFieldValueForHiveIndex(int hiveIndex) + { + return values.get((short) (hiveIndex + 1)); + } + + public ThriftGeneralRow deepCopy() + { + return new ThriftGeneralRow(values); + } + + public void clear() {} + + public Fields fieldForId(int fieldId) + { + return new Fields((short) fieldId, "dummy"); + } + + public Object getFieldValue(Fields field) + { + return values.get(field.thriftId); + } + + public boolean isSet(Fields field) + { + return values.containsKey(field.getThriftFieldId()); + } + public void setFieldValue(Fields field, Object value) { - throw new UnsupportedOperationException("DummyClass.setFieldValue is not supported."); + throw new UnsupportedOperationException("ThriftGeneralRow.setFieldValue is not supported."); } public void write(TProtocol oprot) { - throw new UnsupportedOperationException("DummyClass.write is not supported."); + throw new UnsupportedOperationException("ThriftGeneralRow.write is not supported."); } - public int compareTo(DummyClass other) + public int compareTo(ThriftGeneralRow other) { - throw new UnsupportedOperationException("Dummy.compareTo is not supported."); + throw new UnsupportedOperationException("ThriftGeneralRow.compareTo is not supported."); } } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java similarity index 94% rename from presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveRecordCursor.java rename to presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java index a29e74c9ec46..522cfb3722ff 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveRecordCursor.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java @@ -11,8 +11,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.facebook.presto.hive; +package com.facebook.presto.twitter.hive.thrift; +import com.facebook.presto.hive.HiveColumnHandle; +import com.facebook.presto.hive.HiveType; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.RecordCursor; import com.facebook.presto.spi.block.Block; @@ -23,16 +25,14 @@ import com.facebook.presto.spi.type.Decimals; import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.TypeManager; -import com.facebook.presto.twitter.hive.thrift.DummyClass; import com.google.common.base.Throwables; import io.airlift.log.Logger; import io.airlift.slice.Slice; import io.airlift.slice.Slices; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.common.type.HiveChar; import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.common.type.HiveVarchar; -import org.apache.hadoop.hive.serde2.Deserializer; -import org.apache.hadoop.hive.serde2.SerDeException; import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapred.RecordReader; import org.joda.time.DateTimeZone; @@ -50,7 +50,6 @@ import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveErrorCode.HIVE_CURSOR_ERROR; import static com.facebook.presto.hive.HiveUtil.closeWithSuppression; -import static com.facebook.presto.hive.HiveUtil.getDeserializer; import static com.facebook.presto.hive.HiveUtil.isArrayType; import static com.facebook.presto.hive.HiveUtil.isMapType; import static com.facebook.presto.hive.HiveUtil.isRowType; @@ -86,8 +85,7 @@ class ThriftHiveRecordCursor private final K key; private final V value; - @SuppressWarnings("deprecation") - private final Deserializer deserializer; + private final ThriftGeneralDeserializer deserializer; private final Type[] types; private final HiveType[] hiveTypes; @@ -105,7 +103,7 @@ class ThriftHiveRecordCursor private final DateTimeZone hiveStorageTimeZone; private long completedBytes; - private Object rowData; + private ThriftGeneralRow rowData; private boolean closed; public ThriftHiveRecordCursor( @@ -128,7 +126,8 @@ public ThriftHiveRecordCursor( this.value = recordReader.createValue(); this.hiveStorageTimeZone = hiveStorageTimeZone; - this.deserializer = getDeserializer(splitSchema); + this.deserializer = new ThriftGeneralDeserializer(); + deserializer.initialize(new Configuration(false), splitSchema); int size = columns.size(); @@ -209,7 +208,7 @@ public boolean advanceNextPosition() return true; } - catch (IOException | SerDeException | RuntimeException e) { + catch (IOException | RuntimeException e) { closeWithSuppression(this, e); throw new PrestoException(HIVE_CURSOR_ERROR, e); } @@ -231,8 +230,7 @@ private void parseBooleanColumn(int column) { loaded[column] = true; - DummyClass dummy = (DummyClass) rowData; - Object fieldValue = dummy.values.get((short) hiveIndexs[column]); + Object fieldValue = rowData.getFieldValueForHiveIndex(hiveIndexs[column]); if (fieldValue == null) { nulls[column] = true; @@ -259,8 +257,7 @@ private void parseLongColumn(int column) { loaded[column] = true; - DummyClass dummy = (DummyClass) rowData; - Object fieldValue = dummy.values.get((short) hiveIndexs[column]); + Object fieldValue = rowData.getFieldValueForHiveIndex(hiveIndexs[column]); if (fieldValue == null) { nulls[column] = true; @@ -318,8 +315,7 @@ private void parseDoubleColumn(int column) { loaded[column] = true; - DummyClass dummy = (DummyClass) rowData; - Object fieldValue = dummy.values.get((short) hiveIndexs[column]); + Object fieldValue = rowData.getFieldValueForHiveIndex(hiveIndexs[column]); if (fieldValue == null) { nulls[column] = true; @@ -346,8 +342,7 @@ private void parseStringColumn(int column) { loaded[column] = true; - DummyClass dummy = (DummyClass) rowData; - Object fieldValue = dummy.values.get((short) hiveIndexs[column]); + Object fieldValue = rowData.getFieldValueForHiveIndex(hiveIndexs[column]); if (fieldValue == null) { nulls[column] = true; @@ -393,8 +388,7 @@ private void parseDecimalColumn(int column) { loaded[column] = true; - DummyClass dummy = (DummyClass) rowData; - Object fieldValue = dummy.values.get((short) hiveIndexs[column]); + Object fieldValue = rowData.getFieldValueForHiveIndex(hiveIndexs[column]); if (fieldValue == null) { nulls[column] = true; @@ -430,8 +424,7 @@ private void parseObjectColumn(int column) { loaded[column] = true; - DummyClass dummy = (DummyClass) rowData; - Object fieldValue = dummy.values.get((short) hiveIndexs[column]); + Object fieldValue = rowData.getFieldValueForHiveIndex(hiveIndexs[column]); if (fieldValue == null) { nulls[column] = true; @@ -629,7 +622,7 @@ private static Block serializeStruct(Type type, BlockBuilder builder, Object obj } List typeParameters = type.getTypeParameters(); - Map allValues = ((DummyClass) object).values; + ThriftGeneralRow structData = (ThriftGeneralRow) object; BlockBuilder currentBuilder; if (builder != null) { currentBuilder = builder.beginBlockEntry(); @@ -639,8 +632,8 @@ private static Block serializeStruct(Type type, BlockBuilder builder, Object obj } for (int i = 0; i < typeParameters.size(); i++) { - Object field = allValues.get((short) i); - serializeObject(typeParameters.get(i), currentBuilder, field, hiveStorageTimeZone); + Object fieldValue = structData.getFieldValueForHiveIndex(i); + serializeObject(typeParameters.get(i), currentBuilder, fieldValue, hiveStorageTimeZone); } if (builder != null) { diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveRecordCursorProvider.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursorProvider.java similarity index 87% rename from presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveRecordCursorProvider.java rename to presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursorProvider.java index d3a3edd7bdd6..89eda44db3e8 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/ThriftHiveRecordCursorProvider.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursorProvider.java @@ -11,8 +11,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.facebook.presto.hive; +package com.facebook.presto.twitter.hive.thrift; +import com.facebook.presto.hive.HdfsEnvironment; +import com.facebook.presto.hive.HiveColumnHandle; +import com.facebook.presto.hive.HiveRecordCursorProvider; import com.facebook.presto.spi.ConnectorSession; import com.facebook.presto.spi.RecordCursor; import com.facebook.presto.spi.predicate.TupleDomain; @@ -33,6 +36,7 @@ import java.util.Properties; import java.util.Set; +import static com.facebook.presto.hive.HiveUtil.createRecordReader; import static com.facebook.presto.hive.HiveUtil.getDeserializerClassName; import static java.util.Objects.requireNonNull; @@ -40,7 +44,8 @@ public class ThriftHiveRecordCursorProvider implements HiveRecordCursorProvider { private static final Set THRIFT_SERDE_CLASS_NAMES = ImmutableSet.builder() - .add("com.twitter.elephantbird.hive.serde.ThriftSerDe") + .add("com.facebook.presto.twitter.hive.thrift.ThriftGeneralSerDe") + .add("com.facebook.presto.twitter.hive.thrift.ThriftGeneralDeserializer") .build(); private final HdfsEnvironment hdfsEnvironment; @@ -74,7 +79,7 @@ public Optional createRecordCursor( } else { recordReader = hdfsEnvironment.doAs(session.getUser(), - () -> HiveUtil.createRecordReader(configuration, path, start, length, schema, columns)); + () -> createRecordReader(configuration, path, start, length, schema, columns)); } return Optional.of(new ThriftHiveRecordCursor<>( From d071d79dc8b54ff342d44eb7d3200dfbbd1eb9a6 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Mon, 20 Mar 2017 14:09:35 -0700 Subject: [PATCH 149/331] Set elephant-bird task config to avoid decode errors getting suppressed --- .../presto/twitter/hive/thrift/ThriftGeneralInputFormat.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java index e92560729804..0bc618c8007c 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java @@ -59,6 +59,9 @@ private void initialize(FileSplit split, JobConf job) throws IOException public RecordReader getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException { + job.setBoolean("elephantbird.mapred.input.bad.record.check.only.in.close", false); + job.setFloat("elephantbird.mapred.input.bad.record.threshold", 0.0f); + initialize((FileSplit) split, job); return super.getRecordReader(split, job, reporter); } From 29a21ba54315a8f76b035a3a095a714d9a29ed54 Mon Sep 17 00:00:00 2001 From: dabaitu Date: Tue, 21 Mar 2017 11:52:37 -0700 Subject: [PATCH 150/331] Enable splits for LZO Thrift data --- .../hive/BackgroundHiveSplitLoader.java | 46 +++++++++++++++++++ 1 file changed, 46 insertions(+) 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 1d48e2dc4dc3..5b2452204271 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 @@ -24,6 +24,7 @@ import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.StandardErrorCode; import com.facebook.presto.spi.predicate.TupleDomain; +import com.facebook.presto.twitter.hive.thrift.ThriftGeneralInputFormat; import com.facebook.presto.twitter.hive.util.UgiUtils; import com.google.common.base.Throwables; import com.google.common.collect.AbstractIterator; @@ -368,6 +369,11 @@ private void loadPartition(HivePartitionMetadata partition) if (!buckets.isEmpty()) { int bucketCount = buckets.get(0).getBucketCount(); List list = listAndSortBucketFiles(iterator, bucketCount); + if (inputFormat instanceof ThriftGeneralInputFormat) { + addThriftSplitsToQueue(list, partitionName, schema, partitionKeys, session, effectivePredicate, partition.getColumnCoercions()); + return; + } + List> iteratorList = new ArrayList<>(); for (HiveBucket bucket : buckets) { @@ -427,6 +433,46 @@ private void loadPartition(HivePartitionMetadata partition) fileIterators.addLast(iterator); } + private void addThriftSplitsToQueue( + List files, + String partitionName, + Properties schema, + List partitionKeys, + ConnectorSession session, + TupleDomain effectivePredicate, + Map columnCoercions) + throws IOException + { + for (LocatedFileStatus lfs : files) { + ThriftGeneralInputFormat targetInputFormat = new ThriftGeneralInputFormat(); + Configuration targetConfiguration = hdfsEnvironment.getConfiguration(lfs.getPath()); + JobConf targetJob = new JobConf(targetConfiguration); + targetJob.setInputFormat(ThriftGeneralInputFormat.class); + InputSplit[] targetSplits = targetInputFormat.getSplits(targetJob, 0); + for (InputSplit inputSplit : targetSplits) { + FileSplit split = (FileSplit) inputSplit; + FileSystem targetFilesystem = hdfsEnvironment.getFileSystem(session.getUser(), split.getPath()); + FileStatus file = targetFilesystem.getFileStatus(split.getPath()); + hiveSplitSource.addToQueue(createHiveSplitIterator( + partitionName, + file.getPath().toString(), + targetFilesystem.getFileBlockLocations(file, split.getStart(), split.getLength()), + split.getStart(), + split.getLength(), + schema, + partitionKeys, + false, + session, + OptionalInt.empty(), + effectivePredicate, + columnCoercions)); + if (stopped) { + return; + } + } + } + } + private void addToHiveSplitSourceRoundRobin(List> iteratorList) { while (true) { From b9acafcbc79ee2e50e56a237d46dd5caa3f84e52 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 22 Mar 2017 11:00:13 -0700 Subject: [PATCH 151/331] Add support for thrif id mapping --- .../presto/hive/HiveClientModule.java | 3 + .../thrift/HiveThriftFieldIdResolver.java | 79 +++++++++++++++++++ .../hive/thrift/ThriftFieldIdResolver.java | 23 ++++++ .../twitter/hive/thrift/ThriftGeneralRow.java | 4 +- .../hive/thrift/ThriftHiveRecordCursor.java | 60 ++++++++------ .../ThriftHiveRecordCursorProvider.java | 7 +- 6 files changed, 148 insertions(+), 28 deletions(-) create mode 100644 presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftFieldIdResolver.java 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 dd0406d0d47b..958ffb2207b2 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 @@ -27,6 +27,8 @@ import com.facebook.presto.spi.connector.ConnectorSplitManager; import com.facebook.presto.spi.type.TypeManager; import com.facebook.presto.twitter.hive.PooledHiveMetastoreClientFactory; +import com.facebook.presto.twitter.hive.thrift.HiveThriftFieldIdResolver; +import com.facebook.presto.twitter.hive.thrift.ThriftFieldIdResolver; import com.facebook.presto.twitter.hive.thrift.ThriftHiveRecordCursorProvider; import com.google.inject.Binder; import com.google.inject.Module; @@ -89,6 +91,7 @@ public void configure(Binder binder) binder.bind(NodeManager.class).toInstance(nodeManager); binder.bind(TypeManager.class).toInstance(typeManager); binder.bind(PageIndexerFactory.class).toInstance(pageIndexerFactory); + binder.bind(ThriftFieldIdResolver.class).toInstance(new HiveThriftFieldIdResolver()); Multibinder recordCursorProviderBinder = Multibinder.newSetBinder(binder, HiveRecordCursorProvider.class); recordCursorProviderBinder.addBinding().to(ParquetRecordCursorProvider.class).in(Scopes.SINGLETON); diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java new file mode 100644 index 000000000000..f1ca6442a07b --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java @@ -0,0 +1,79 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.twitter.hive.thrift; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.airlift.log.Logger; + +import java.util.Properties; + +public class HiveThriftFieldIdResolver + implements ThriftFieldIdResolver +{ + private static final Logger log = Logger.get(HiveThriftFieldIdResolver.class); + private static final ObjectMapper objectMapper = new ObjectMapper(); + public static final String THRIFT_FIELD_ID_JSON = "thrift.field.id.json"; + private final JsonNode root; + + public HiveThriftFieldIdResolver() + { + this.root = null; + } + + public HiveThriftFieldIdResolver(JsonNode root) + { + this.root = root; + } + + public ThriftFieldIdResolver initialize(Properties schema) + { + String jsonData = schema.getProperty(THRIFT_FIELD_ID_JSON); + try { + return new HiveThriftFieldIdResolver(objectMapper.readTree(jsonData)); + } + catch (Exception e) { + log.debug("Got an exception %s in initialize, schema: %s", e.getMessage(), schema); + return new HiveThriftFieldIdResolver(); + } + } + + public ThriftFieldIdResolver getNestedResolver(int hiveIndex) + { + try { + return new HiveThriftFieldIdResolver(root.get(String.valueOf(hiveIndex))); + } + catch (Exception e) { + log.debug("Got an exception %s in getNestedResolver, root: %s, want the hiveIndex: %s", e.getMessage(), root, hiveIndex); + return new HiveThriftFieldIdResolver(); + } + } + + public short getThriftId(int hiveIndex) + { + try { + return (short) root.get(String.valueOf(hiveIndex)).get("id").asInt(); + } + catch (Exception e) { + log.debug("Got an exception %s in getThriftId, root: %s, want the hiveIndex: %s", e.getMessage(), root, hiveIndex); + return (short) (hiveIndex + 1); + } + } + + @Override + public String toString() + { + return "HiveThriftFieldIdResolver with root:" + root; + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftFieldIdResolver.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftFieldIdResolver.java new file mode 100644 index 000000000000..068e485e8989 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftFieldIdResolver.java @@ -0,0 +1,23 @@ +/* + * 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.twitter.hive.thrift; + +import java.util.Properties; + +public interface ThriftFieldIdResolver +{ + ThriftFieldIdResolver initialize(Properties schema); + ThriftFieldIdResolver getNestedResolver(int hiveIndex); + short getThriftId(int hiveIndex); +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralRow.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralRow.java index 689298606606..d332b849565e 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralRow.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralRow.java @@ -151,9 +151,9 @@ private Object readMap(TProtocol iprot) throws TException return mapValue; } - public Object getFieldValueForHiveIndex(int hiveIndex) + public Object getFieldValueForThriftId(short thriftId) { - return values.get((short) (hiveIndex + 1)); + return values.get(thriftId); } public ThriftGeneralRow deepCopy() diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java index 522cfb3722ff..7f7c1313ef57 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java @@ -90,6 +90,7 @@ class ThriftHiveRecordCursor private final Type[] types; private final HiveType[] hiveTypes; private final int[] hiveIndexs; + private final short[] thriftIds; private final boolean[] loaded; private final boolean[] booleans; @@ -102,6 +103,8 @@ class ThriftHiveRecordCursor private final long totalBytes; private final DateTimeZone hiveStorageTimeZone; + private final ThriftFieldIdResolver thriftFieldIdResolver; + private long completedBytes; private ThriftGeneralRow rowData; private boolean closed; @@ -112,19 +115,22 @@ public ThriftHiveRecordCursor( Properties splitSchema, List columns, DateTimeZone hiveStorageTimeZone, - TypeManager typeManager) + TypeManager typeManager, + ThriftFieldIdResolver thriftFieldIdResolver) { requireNonNull(recordReader, "recordReader is null"); checkArgument(totalBytes >= 0, "totalBytes is negative"); requireNonNull(splitSchema, "splitSchema is null"); requireNonNull(columns, "columns is null"); requireNonNull(hiveStorageTimeZone, "hiveStorageTimeZone is null"); + requireNonNull(thriftFieldIdResolver, "thriftFieldIdResolver is null"); this.recordReader = recordReader; this.totalBytes = totalBytes; this.key = recordReader.createKey(); this.value = recordReader.createValue(); this.hiveStorageTimeZone = hiveStorageTimeZone; + this.thriftFieldIdResolver = thriftFieldIdResolver; this.deserializer = new ThriftGeneralDeserializer(); deserializer.initialize(new Configuration(false), splitSchema); @@ -134,6 +140,7 @@ public ThriftHiveRecordCursor( this.types = new Type[size]; this.hiveTypes = new HiveType[size]; this.hiveIndexs = new int[size]; + this.thriftIds = new short[size]; this.loaded = new boolean[size]; this.booleans = new boolean[size]; @@ -151,6 +158,7 @@ public ThriftHiveRecordCursor( types[i] = typeManager.getType(column.getTypeSignature()); hiveTypes[i] = column.getHiveType(); hiveIndexs[i] = column.getHiveColumnIndex(); + thriftIds[i] = thriftFieldIdResolver.getThriftId(hiveIndexs[i]); } } @@ -230,7 +238,7 @@ private void parseBooleanColumn(int column) { loaded[column] = true; - Object fieldValue = rowData.getFieldValueForHiveIndex(hiveIndexs[column]); + Object fieldValue = rowData.getFieldValueForThriftId(thriftIds[column]); if (fieldValue == null) { nulls[column] = true; @@ -257,7 +265,7 @@ private void parseLongColumn(int column) { loaded[column] = true; - Object fieldValue = rowData.getFieldValueForHiveIndex(hiveIndexs[column]); + Object fieldValue = rowData.getFieldValueForThriftId(thriftIds[column]); if (fieldValue == null) { nulls[column] = true; @@ -315,7 +323,7 @@ private void parseDoubleColumn(int column) { loaded[column] = true; - Object fieldValue = rowData.getFieldValueForHiveIndex(hiveIndexs[column]); + Object fieldValue = rowData.getFieldValueForThriftId(thriftIds[column]); if (fieldValue == null) { nulls[column] = true; @@ -342,7 +350,7 @@ private void parseStringColumn(int column) { loaded[column] = true; - Object fieldValue = rowData.getFieldValueForHiveIndex(hiveIndexs[column]); + Object fieldValue = rowData.getFieldValueForThriftId(thriftIds[column]); if (fieldValue == null) { nulls[column] = true; @@ -388,7 +396,7 @@ private void parseDecimalColumn(int column) { loaded[column] = true; - Object fieldValue = rowData.getFieldValueForHiveIndex(hiveIndexs[column]); + Object fieldValue = rowData.getFieldValueForThriftId(thriftIds[column]); if (fieldValue == null) { nulls[column] = true; @@ -424,13 +432,14 @@ private void parseObjectColumn(int column) { loaded[column] = true; - Object fieldValue = rowData.getFieldValueForHiveIndex(hiveIndexs[column]); + Object fieldValue = rowData.getFieldValueForThriftId(thriftIds[column]); if (fieldValue == null) { nulls[column] = true; } else { - objects[column] = getBlockObject(types[column], fieldValue, hiveStorageTimeZone); + ThriftFieldIdResolver resolver = thriftFieldIdResolver.getNestedResolver(hiveIndexs[column]); + objects[column] = getBlockObject(types[column], resolver, fieldValue, hiveStorageTimeZone); nulls[column] = false; } } @@ -520,30 +529,30 @@ public void close() } } - private static Block getBlockObject(Type type, Object object, DateTimeZone hiveStorageTimeZone) + private static Block getBlockObject(Type type, ThriftFieldIdResolver resolver, Object object, DateTimeZone hiveStorageTimeZone) { - return requireNonNull(serializeObject(type, null, object, hiveStorageTimeZone), "serialized result is null"); + return requireNonNull(serializeObject(type, resolver, null, object, hiveStorageTimeZone), "serialized result is null"); } - private static Block serializeObject(Type type, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) + private static Block serializeObject(Type type, ThriftFieldIdResolver resolver, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) { if (!isStructuralType(type)) { - serializePrimitive(type, builder, object, hiveStorageTimeZone); + serializePrimitive(type, resolver, builder, object, hiveStorageTimeZone); return null; } else if (isArrayType(type)) { - return serializeList(type, builder, object, hiveStorageTimeZone); + return serializeList(type, resolver, builder, object, hiveStorageTimeZone); } else if (isMapType(type)) { - return serializeMap(type, builder, object, hiveStorageTimeZone); + return serializeMap(type, resolver, builder, object, hiveStorageTimeZone); } else if (isRowType(type)) { - return serializeStruct(type, builder, object, hiveStorageTimeZone); + return serializeStruct(type, resolver, builder, object, hiveStorageTimeZone); } throw new RuntimeException("Unknown object type: " + type); } - private static Block serializeList(Type type, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) + private static Block serializeList(Type type, ThriftFieldIdResolver resolver, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) { List list = (List) object; if (list == null) { @@ -554,6 +563,7 @@ private static Block serializeList(Type type, BlockBuilder builder, Object objec List typeParameters = type.getTypeParameters(); checkArgument(typeParameters.size() == 1, "list must have exactly 1 type parameter"); Type elementType = typeParameters.get(0); + ThriftFieldIdResolver elementResolver = resolver.getNestedResolver(0); BlockBuilder currentBuilder; if (builder != null) { currentBuilder = builder.beginBlockEntry(); @@ -563,7 +573,7 @@ private static Block serializeList(Type type, BlockBuilder builder, Object objec } for (Object element : list) { - serializeObject(elementType, currentBuilder, element, hiveStorageTimeZone); + serializeObject(elementType, elementResolver, currentBuilder, element, hiveStorageTimeZone); } if (builder != null) { @@ -576,7 +586,7 @@ private static Block serializeList(Type type, BlockBuilder builder, Object objec } } - private static Block serializeMap(Type type, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) + private static Block serializeMap(Type type, ThriftFieldIdResolver resolver, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) { Map map = (Map) object; if (map == null) { @@ -588,6 +598,8 @@ private static Block serializeMap(Type type, BlockBuilder builder, Object object checkArgument(typeParameters.size() == 2, "map must have exactly 2 type parameter"); Type keyType = typeParameters.get(0); Type valueType = typeParameters.get(1); + ThriftFieldIdResolver keyResolver = resolver.getNestedResolver(0); + ThriftFieldIdResolver valueResolver = resolver.getNestedResolver(1); BlockBuilder currentBuilder; if (builder != null) { currentBuilder = builder.beginBlockEntry(); @@ -599,8 +611,8 @@ private static Block serializeMap(Type type, BlockBuilder builder, Object object for (Map.Entry entry : map.entrySet()) { // Hive skips map entries with null keys if (entry.getKey() != null) { - serializeObject(keyType, currentBuilder, entry.getKey(), hiveStorageTimeZone); - serializeObject(valueType, currentBuilder, entry.getValue(), hiveStorageTimeZone); + serializeObject(keyType, keyResolver, currentBuilder, entry.getKey(), hiveStorageTimeZone); + serializeObject(valueType, valueResolver, currentBuilder, entry.getValue(), hiveStorageTimeZone); } } @@ -614,7 +626,7 @@ private static Block serializeMap(Type type, BlockBuilder builder, Object object } } - private static Block serializeStruct(Type type, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) + private static Block serializeStruct(Type type, ThriftFieldIdResolver resolver, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) { if (object == null) { requireNonNull(builder, "parent builder is null").appendNull(); @@ -632,8 +644,8 @@ private static Block serializeStruct(Type type, BlockBuilder builder, Object obj } for (int i = 0; i < typeParameters.size(); i++) { - Object fieldValue = structData.getFieldValueForHiveIndex(i); - serializeObject(typeParameters.get(i), currentBuilder, fieldValue, hiveStorageTimeZone); + Object fieldValue = structData.getFieldValueForThriftId(resolver.getThriftId(i)); + serializeObject(typeParameters.get(i), resolver.getNestedResolver(i), currentBuilder, fieldValue, hiveStorageTimeZone); } if (builder != null) { @@ -646,7 +658,7 @@ private static Block serializeStruct(Type type, BlockBuilder builder, Object obj } } - private static void serializePrimitive(Type type, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) + private static void serializePrimitive(Type type, ThriftFieldIdResolver resolver, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) { requireNonNull(builder, "parent builder is null"); 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 89eda44db3e8..1e10b5e8a1c4 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 @@ -48,11 +48,13 @@ public class ThriftHiveRecordCursorProvider .add("com.facebook.presto.twitter.hive.thrift.ThriftGeneralDeserializer") .build(); private final HdfsEnvironment hdfsEnvironment; + private final ThriftFieldIdResolver thriftFieldIdResolver; @Inject - public ThriftHiveRecordCursorProvider(HdfsEnvironment hdfsEnvironment) + public ThriftHiveRecordCursorProvider(HdfsEnvironment hdfsEnvironment, ThriftFieldIdResolver thriftFieldIdResolver) { this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); + this.thriftFieldIdResolver = requireNonNull(thriftFieldIdResolver, "thriftFieldIdResolver is null"); } @Override @@ -88,7 +90,8 @@ public Optional createRecordCursor( schema, columns, hiveStorageTimeZone, - typeManager)); + typeManager, + thriftFieldIdResolver.initialize(schema))); } @SuppressWarnings("unchecked") From 63fca50fb853947c63f7f873cecba9dabbd04d92 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 22 Mar 2017 15:03:25 -0700 Subject: [PATCH 152/331] Better Naming: ThriftGeneralRow -> ThriftGenericRow --- .../thrift/ThriftGeneralDeserializer.java | 6 ++--- ...tGeneralRow.java => ThriftGenericRow.java} | 22 +++++++++---------- .../hive/thrift/ThriftHiveRecordCursor.java | 4 ++-- 3 files changed, 16 insertions(+), 16 deletions(-) rename presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/{ThriftGeneralRow.java => ThriftGenericRow.java} (89%) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralDeserializer.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralDeserializer.java index cddd4f1f85e5..49ceeeac8957 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralDeserializer.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralDeserializer.java @@ -26,7 +26,7 @@ public class ThriftGeneralDeserializer { - private static final String REQUIRED_SERIALIZATION_CLASS = "com.facebook.presto.twitter.hive.thrift.ThriftGeneralRow"; + private static final String REQUIRED_SERIALIZATION_CLASS = "com.facebook.presto.twitter.hive.thrift.ThriftGenericRow"; public void initialize(Configuration conf, Properties properties) { @@ -36,9 +36,9 @@ public void initialize(Configuration conf, Properties properties) return; } - public ThriftGeneralRow deserialize(Writable writable) + public ThriftGenericRow deserialize(Writable writable) { checkCondition(writable instanceof ThriftWritable, HIVE_UNKNOWN_ERROR, "Not an instance of ThriftWritable: " + writable); - return (ThriftGeneralRow) ((ThriftWritable) writable).get(); + return (ThriftGenericRow) ((ThriftWritable) writable).get(); } } diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralRow.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java similarity index 89% rename from presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralRow.java rename to presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java index d332b849565e..ce71c38268c3 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralRow.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java @@ -30,14 +30,14 @@ import java.util.List; import java.util.Map; -public class ThriftGeneralRow implements TBase +public class ThriftGenericRow implements TBase { - private static final Logger log = Logger.get(ThriftGeneralRow.class); + private static final Logger log = Logger.get(ThriftGenericRow.class); private final Map values = new HashMap<>(); - public ThriftGeneralRow() {} + public ThriftGenericRow() {} - public ThriftGeneralRow(Map values) + public ThriftGenericRow(Map values) { this.values.putAll(values); } @@ -113,7 +113,7 @@ private Object readElem(TProtocol iprot, byte type) throws TException private Object readStruct(TProtocol iprot) throws TException { - ThriftGeneralRow elem = new ThriftGeneralRow(); + ThriftGenericRow elem = new ThriftGenericRow(); elem.read(iprot); return elem; } @@ -156,9 +156,9 @@ public Object getFieldValueForThriftId(short thriftId) return values.get(thriftId); } - public ThriftGeneralRow deepCopy() + public ThriftGenericRow deepCopy() { - return new ThriftGeneralRow(values); + return new ThriftGenericRow(values); } public void clear() {} @@ -180,16 +180,16 @@ public boolean isSet(Fields field) public void setFieldValue(Fields field, Object value) { - throw new UnsupportedOperationException("ThriftGeneralRow.setFieldValue is not supported."); + throw new UnsupportedOperationException("ThriftGenericRow.setFieldValue is not supported."); } public void write(TProtocol oprot) { - throw new UnsupportedOperationException("ThriftGeneralRow.write is not supported."); + throw new UnsupportedOperationException("ThriftGenericRow.write is not supported."); } - public int compareTo(ThriftGeneralRow other) + public int compareTo(ThriftGenericRow other) { - throw new UnsupportedOperationException("ThriftGeneralRow.compareTo is not supported."); + throw new UnsupportedOperationException("ThriftGenericRow.compareTo is not supported."); } } diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java index 7f7c1313ef57..1b3d42c84e7a 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java @@ -106,7 +106,7 @@ class ThriftHiveRecordCursor private final ThriftFieldIdResolver thriftFieldIdResolver; private long completedBytes; - private ThriftGeneralRow rowData; + private ThriftGenericRow rowData; private boolean closed; public ThriftHiveRecordCursor( @@ -634,7 +634,7 @@ private static Block serializeStruct(Type type, ThriftFieldIdResolver resolver, } List typeParameters = type.getTypeParameters(); - ThriftGeneralRow structData = (ThriftGeneralRow) object; + ThriftGenericRow structData = (ThriftGenericRow) object; BlockBuilder currentBuilder; if (builder != null) { currentBuilder = builder.beginBlockEntry(); From e90d854b1a2c35f0820c3d322b0cf14e11e73964 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 23 Mar 2017 17:52:38 -0700 Subject: [PATCH 153/331] Support splittable lzo --- presto-hive/pom.xml | 11 ++- .../hive/BackgroundHiveSplitLoader.java | 45 +++++++---- .../hive/thrift/ThriftGeneralInputFormat.java | 75 ++++++++++++++++++- .../ThriftHiveRecordCursorProvider.java | 40 +--------- 4 files changed, 109 insertions(+), 62 deletions(-) diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index e76118853d35..34b1cacb4c33 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -250,6 +250,11 @@ runtime + + org.anarres.lzo + lzo-hadoop + + com.facebook.presto @@ -320,12 +325,6 @@ test - - org.anarres.lzo - lzo-hadoop - test - - com.facebook.presto 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 5b2452204271..da6f493ddff5 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 @@ -32,6 +32,7 @@ import com.google.common.collect.Iterators; import com.google.common.collect.PeekingIterator; import com.google.common.io.CharStreams; +import io.airlift.log.Logger; import io.airlift.units.DataSize; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BlockLocation; @@ -90,6 +91,7 @@ public class BackgroundHiveSplitLoader implements HiveSplitLoader { private static final String CORRUPT_BUCKETING = "Hive table is corrupt. It is declared as being bucketed, but the files do not match the bucketing declaration."; + private static final Logger log = Logger.get(BackgroundHiveSplitLoader.class); public static final CompletableFuture COMPLETED_FUTURE = CompletableFuture.completedFuture(null); @@ -366,14 +368,15 @@ private void loadPartition(HivePartitionMetadata partition) // If only one bucket could match: load that one file HiveFileIterator iterator = new HiveFileIterator(path, fs, directoryLister, namenodeStats, partitionName, inputFormat, schema, partitionKeys, effectivePredicate, partition.getColumnCoercions()); + + if (inputFormat instanceof ThriftGeneralInputFormat) { + addLzoThriftSplitsToQueue(iterator, partitionName, schema, partitionKeys, session, effectivePredicate, partition.getColumnCoercions()); + return; + } + if (!buckets.isEmpty()) { int bucketCount = buckets.get(0).getBucketCount(); List list = listAndSortBucketFiles(iterator, bucketCount); - if (inputFormat instanceof ThriftGeneralInputFormat) { - addThriftSplitsToQueue(list, partitionName, schema, partitionKeys, session, effectivePredicate, partition.getColumnCoercions()); - return; - } - List> iteratorList = new ArrayList<>(); for (HiveBucket bucket : buckets) { @@ -433,8 +436,8 @@ private void loadPartition(HivePartitionMetadata partition) fileIterators.addLast(iterator); } - private void addThriftSplitsToQueue( - List files, + private void addLzoThriftSplitsToQueue( + HiveFileIterator hiveFileIterator, String partitionName, Properties schema, List partitionKeys, @@ -443,19 +446,29 @@ private void addThriftSplitsToQueue( Map columnCoercions) throws IOException { - for (LocatedFileStatus lfs : files) { - ThriftGeneralInputFormat targetInputFormat = new ThriftGeneralInputFormat(); - Configuration targetConfiguration = hdfsEnvironment.getConfiguration(lfs.getPath()); + if (bucketHandle.isPresent()) { + throw new PrestoException(StandardErrorCode.NOT_SUPPORTED, "Bucketed table in ThriftGeneralInputFormat is not yet supported"); + } + List> iteratorList = new ArrayList<>(); + + while (hiveFileIterator.hasNext()) { + LocatedFileStatus file = hiveFileIterator.next(); + if (!ThriftGeneralInputFormat.lzoSuffixFilter.accept(file.getPath())) { + continue; + } + + Configuration targetConfiguration = hdfsEnvironment.getConfiguration(file.getPath()); JobConf targetJob = new JobConf(targetConfiguration); - targetJob.setInputFormat(ThriftGeneralInputFormat.class); - InputSplit[] targetSplits = targetInputFormat.getSplits(targetJob, 0); + FileSystem indexFilesystem = hdfsEnvironment.getFileSystem(session.getUser(), ThriftGeneralInputFormat.getLzoIndexPath(file.getPath())); + InputSplit[] targetSplits = ThriftGeneralInputFormat.getLzoSplits(targetJob, file, indexFilesystem, remainingInitialSplits, maxInitialSplitSize, maxSplitSize); + log.debug("For file at %s, get number of splits %s", file.getPath(), targetSplits.length); for (InputSplit inputSplit : targetSplits) { FileSplit split = (FileSplit) inputSplit; FileSystem targetFilesystem = hdfsEnvironment.getFileSystem(session.getUser(), split.getPath()); - FileStatus file = targetFilesystem.getFileStatus(split.getPath()); - hiveSplitSource.addToQueue(createHiveSplitIterator( + FileStatus targetFile = targetFilesystem.getFileStatus(split.getPath()); + iteratorList.add(createHiveSplitIterator( partitionName, - file.getPath().toString(), + targetFile.getPath().toString(), targetFilesystem.getFileBlockLocations(file, split.getStart(), split.getLength()), split.getStart(), split.getLength(), @@ -471,6 +484,8 @@ private void addThriftSplitsToQueue( } } } + + addToHiveSplitSourceRoundRobin(iteratorList); } private void addToHiveSplitSourceRoundRobin(List> iteratorList) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java index 0bc618c8007c..48c0f05bb0b3 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java @@ -13,10 +13,16 @@ */ package com.facebook.presto.twitter.hive.thrift; +import com.hadoop.compression.lzo.LzoIndex; import com.twitter.elephantbird.mapred.input.DeprecatedFileInputFormatWrapper; import com.twitter.elephantbird.mapreduce.input.MultiInputFormat; import com.twitter.elephantbird.mapreduce.io.BinaryWritable; import com.twitter.elephantbird.util.TypeRef; +import io.airlift.units.DataSize; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.mapred.InputSplit; @@ -25,9 +31,13 @@ import org.apache.hadoop.mapred.Reporter; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; import static com.facebook.presto.hive.HiveErrorCode.HIVE_INVALID_METADATA; import static com.facebook.presto.hive.HiveUtil.checkCondition; +import static java.lang.Math.toIntExact; import static org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS; /** @@ -36,6 +46,14 @@ @SuppressWarnings("deprecation") public class ThriftGeneralInputFormat extends DeprecatedFileInputFormatWrapper { + public static final PathFilter lzoSuffixFilter = new PathFilter() { + @Override + public boolean accept(Path path) + { + return path.toString().endsWith(".lzo"); + } + }; + public ThriftGeneralInputFormat() { super(new MultiInputFormat()); @@ -56,8 +74,11 @@ private void initialize(FileSplit split, JobConf job) throws IOException } @Override - public RecordReader getRecordReader(InputSplit split, JobConf job, - Reporter reporter) throws IOException + public RecordReader getRecordReader( + InputSplit split, + JobConf job, + Reporter reporter) + throws IOException { job.setBoolean("elephantbird.mapred.input.bad.record.check.only.in.close", false); job.setFloat("elephantbird.mapred.input.bad.record.threshold", 0.0f); @@ -65,4 +86,54 @@ public RecordReader getRecordReader(InputSplit spl initialize((FileSplit) split, job); return super.getRecordReader(split, job, reporter); } + + public static Path getLzoIndexPath(Path lzoPath) + { + return lzoPath.suffix(LzoIndex.LZO_INDEX_SUFFIX); + } + + public static InputSplit[] getLzoSplits( + JobConf job, + LocatedFileStatus file) + throws IOException + { + InputSplit[] splits = new InputSplit[1]; + splits[0] = new FileSplit(file.getPath(), 0, file.getLen(), job); + return splits; + } + + public static InputSplit[] getLzoSplits( + JobConf job, + LocatedFileStatus file, + FileSystem indexFilesystem, + AtomicInteger remainingInitialSplits, + DataSize maxInitialSplitSize, + DataSize maxSplitSize) + throws IOException + { + LzoIndex index = LzoIndex.readIndex(indexFilesystem, file.getPath()); + if (index.isEmpty()) { + return getLzoSplits(job, file); + } + + List splits = new ArrayList<>(); + long chunkOffset = 0; + while (chunkOffset < file.getLen()) { + long targetChunkSize; + if (remainingInitialSplits.decrementAndGet() >= 0) { + targetChunkSize = maxInitialSplitSize.toBytes(); + } + else { + long maxBytes = maxSplitSize.toBytes(); + int chunks = toIntExact((long) Math.ceil((file.getLen() - chunkOffset) * 1.0 / maxBytes)); + targetChunkSize = (long) Math.ceil((file.getLen() - chunkOffset) * 1.0 / chunks); + } + long chunkEnd = index.alignSliceEndToIndex(chunkOffset + targetChunkSize, file.getLen()); + + splits.add(new FileSplit(file.getPath(), chunkOffset, chunkEnd, job)); + chunkOffset = chunkEnd; + } + + return splits.toArray(new InputSplit[0]); + } } 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 1e10b5e8a1c4..841927816b6b 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 @@ -23,14 +23,12 @@ import com.google.common.collect.ImmutableSet; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapred.RecordReader; import org.joda.time.DateTimeZone; import javax.inject.Inject; -import java.io.IOException; import java.util.List; import java.util.Optional; import java.util.Properties; @@ -75,14 +73,8 @@ public Optional createRecordCursor( return Optional.empty(); } - RecordReader recordReader; - if (path.toString().endsWith(".index")) { - recordReader = new DummyRecordReader(); - } - else { - recordReader = hdfsEnvironment.doAs(session.getUser(), + RecordReader recordReader = hdfsEnvironment.doAs(session.getUser(), () -> createRecordReader(configuration, path, start, length, schema, columns)); - } return Optional.of(new ThriftHiveRecordCursor<>( genericRecordReader(recordReader), @@ -99,34 +91,4 @@ public Optional createRecordCursor( { return (RecordReader) recordReader; } - - private static final class DummyRecordReader implements RecordReader - { - public boolean next(K key, V value) throws IOException - { - return false; - } - - public K createKey() - { - return null; - } - - public V createValue() - { - return null; - } - - public long getPos() throws IOException - { - return 0; - } - - public float getProgress() throws IOException - { - return 0; - } - - public void close() throws IOException {} - } } From 6c56bf60a3818b56854b3e662156566665058df9 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 28 Mar 2017 16:22:37 -0700 Subject: [PATCH 154/331] Remove tw version --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 46 files changed, 48 insertions(+), 48 deletions(-) diff --git a/pom.xml b/pom.xml index 9edf77cbd96d..15c65c65d0fe 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.166-tw-0.31 + 0.170 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 18e3623f526c..6e3629875413 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 58affc56492a..0cc2801e21e0 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 25297a2168ce..3ea6792c2f7e 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 1dd02f442460..f332ecbb2bf7 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 5b604c0e596d..0498aff8f23e 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 2af600963a95..277755e3bc1d 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.166-tw-0.31 + 0.170 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 2eb2d132b558..ee504edadb8f 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 5913798b0c46..a04c03eea80b 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index f41c9f9cf071..80ab1204c3ed 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 332f17205c08..2674a4e51197 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 79ad3bc2263a..1f54b81602a7 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 16631836a091..90b58b905f4a 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 4a42e93c028f..ea162868ed93 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index c956db424e73..bb9c24a6cf13 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 3cd2512c6142..b71112717cec 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 36a5e0ac31bc..d74787600554 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 46b4554e9bd6..b0c33b64e979 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index f6ae9a5b7d52..7d8b444e1864 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 666d8e4c7618..3b674f21cc11 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index e5aa9c5bb065..f3ef0bd0e371 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index eb225cb0c65e..7aefbe02da2b 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index a4e36fd43600..63a4fc2e555c 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 478a88247f11..4300e72374d0 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-main diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 93f880d9ed09..e94ee16cd7b4 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 96ed8deb8e14..ed61cd73b96b 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 5303c81d463b..0353221c4aa2 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 3c22242c260f..7c95670a2c4e 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index d02a3b73bcbc..40c8722dab1b 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 6e201b5e7ca4..3b5d1c8c22e7 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 2bcd90ab72ef..cdfd0c8276ec 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 60e80602b5f3..31d6defada66 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.166-tw-0.31 + 0.170 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 84b38849fc47..a78f6229488e 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index b23698c33e0f..09a858dba7fb 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 0eb4f12aef96..6dd3f95be9b8 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 537a54cc67fd..3e7bbd615c53 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index fb4eba7ded29..e667e7bb81e9 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index b6aefac4915b..22f853233b3e 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 10ba26bd68c3..ac1af80c089a 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 7fa5115082c3..6f92dc2be827 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index f55bbe875ada..833558c5c9fa 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 56635af12836..eeaec4ba6c19 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 6525906426c0..148dd9b9ef97 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.166-tw-0.31 + 0.170 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index f693896db141..55bcab5ecc26 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 4430c468ccf2..c8e3d7f04993 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 054408f8a09e..e7c32cbd302b 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.166-tw-0.31 + 0.170 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.166-tw-0.31 + 0.170 provided From 7f4f700e1c8bb5f4bfa8a4cdd86300b14022d475 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 28 Mar 2017 17:12:05 -0700 Subject: [PATCH 155/331] Release 0.170-tw-0.32 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 47 files changed, 49 insertions(+), 49 deletions(-) diff --git a/pom.xml b/pom.xml index 04ce9c22d10c..0c4d8d63a228 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.170 + 0.170-tw-0.32 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index b5a1a888dcf9..30fe8b9165f3 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 0cc2801e21e0..d7f7024682e3 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 3ea6792c2f7e..31680c745445 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index f332ecbb2bf7..ecf712772755 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 0498aff8f23e..362b1f8f3241 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index b8792b0e10c0..e922fcbb916a 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.170 + 0.170-tw-0.32 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index ee504edadb8f..c2b8f0c378eb 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index a04c03eea80b..9b0d4f9f79d8 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 5e6ef297c9e4..289cad7109fa 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 760de8f591b1..4e1718d8c8a4 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 1f54b81602a7..2f5114c3a4a7 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 90b58b905f4a..61521de2c57c 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index ea162868ed93..9a6a0724f65f 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index bb9c24a6cf13..e31ee043a73b 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index b71112717cec..2e01b0fd9bcd 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index d74787600554..b18f5fd4d701 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index b0c33b64e979..4cdfbe145f3d 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 5426e187c858..afc7eccb483a 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 3b674f21cc11..443e93bfa2bb 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index f3ef0bd0e371..7dc9f4e215ad 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 7aefbe02da2b..22b22b95232f 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 63a4fc2e555c..ceecc3da62a3 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index bfe9361c3c6a..1e4125da4f11 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-main diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 1319cde588c9..2a634850be7f 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index e94ee16cd7b4..ac06d2ea198f 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index ed61cd73b96b..062ee8c05e8f 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 0353221c4aa2..2a9879c22f1c 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index fdc4adff30ea..88d5a99cfe79 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 40c8722dab1b..8c6fae50a5a7 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 3b5d1c8c22e7..ddf520bebb66 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index cdfd0c8276ec..1dba94f35a14 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index b79c6cdd13b8..2c837b379e1d 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.170 + 0.170-tw-0.32 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index a78f6229488e..4f83793ca631 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index db0fb4c442e0..00f49ad4d0db 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 6dd3f95be9b8..6db7c2373423 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 3e7bbd615c53..974ffffabd0a 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index e667e7bb81e9..a6a641bdab23 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 22f853233b3e..a2b420c7465c 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index ac1af80c089a..5c76a7a23ba2 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 6f92dc2be827..b2853ac43c50 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 833558c5c9fa..ef8c6726398d 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index eeaec4ba6c19..0d058bb07f73 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 148dd9b9ef97..0420d4fe046a 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.170 + 0.170-tw-0.32 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 55bcab5ecc26..dfd00558a2a5 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index c8e3d7f04993..f3d599d677f4 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index e7c32cbd302b..0ce3de21ca04 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170 + 0.170-tw-0.32 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.170 + 0.170-tw-0.32 provided From 433cb8483909e8a24600254a5aea884b47a8c793 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 30 Mar 2017 23:56:20 -0700 Subject: [PATCH 156/331] Fix missed method --- .../presto/twitter/hive/thrift/ThriftHiveRecordCursor.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java index 1b3d42c84e7a..898b7a28ad1c 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java @@ -54,7 +54,6 @@ import static com.facebook.presto.hive.HiveUtil.isMapType; import static com.facebook.presto.hive.HiveUtil.isRowType; import static com.facebook.presto.hive.HiveUtil.isStructuralType; -import static com.facebook.presto.hive.util.Types.checkType; import static com.facebook.presto.spi.type.BigintType.BIGINT; import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; import static com.facebook.presto.spi.type.Chars.isCharType; @@ -699,7 +698,7 @@ else if (TIMESTAMP.equals(type)) { } else if (type instanceof DecimalType) { HiveDecimal decimal = (HiveDecimal) object; - DecimalType decimalType = checkType(type, DecimalType.class, "type"); + DecimalType decimalType = (DecimalType) type; BigInteger unscaledDecimal = rescale(decimal.unscaledValue(), decimal.scale(), decimalType.getScale()); if (decimalType.isShort()) { decimalType.writeLong(builder, unscaledDecimal.longValue()); From f07187c1df2fa30c6bc83e55e29bb88c489e6c07 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 31 Mar 2017 17:10:30 -0700 Subject: [PATCH 157/331] Support skip unused top level column --- presto-hive/pom.xml | 6 ++++ .../thrift/ThriftGeneralDeserializer.java | 12 +++++-- .../twitter/hive/thrift/ThriftGenericRow.java | 35 ++++++++++++++++++- .../hive/thrift/ThriftHiveRecordCursor.java | 2 +- 4 files changed, 51 insertions(+), 4 deletions(-) diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 7c7f871d6e04..370c237b83bf 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -245,6 +245,12 @@ runtime + + commons-lang + commons-lang + 2.4 + + org.anarres.lzo lzo-hadoop diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralDeserializer.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralDeserializer.java index 49ceeeac8957..a2591fad8a62 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralDeserializer.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralDeserializer.java @@ -16,6 +16,7 @@ import com.twitter.elephantbird.mapreduce.io.ThriftWritable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Writable; +import org.apache.thrift.TException; import java.util.Properties; @@ -36,9 +37,16 @@ public void initialize(Configuration conf, Properties properties) return; } - public ThriftGenericRow deserialize(Writable writable) + public ThriftGenericRow deserialize(Writable writable, short[] thriftIds) { checkCondition(writable instanceof ThriftWritable, HIVE_UNKNOWN_ERROR, "Not an instance of ThriftWritable: " + writable); - return (ThriftGenericRow) ((ThriftWritable) writable).get(); + ThriftGenericRow row = (ThriftGenericRow) ((ThriftWritable) writable).get(); + try { + row.parse(thriftIds); + } + catch (TException e) { + throw new IllegalStateException("Generic row failed to parse values", e); + } + return row; } } diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java index ce71c38268c3..2d911b09061f 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java @@ -14,9 +14,11 @@ package com.facebook.presto.twitter.hive.thrift; import io.airlift.log.Logger; +import org.apache.commons.lang.ArrayUtils; import org.apache.thrift.TBase; import org.apache.thrift.TException; import org.apache.thrift.TFieldIdEnum; +import org.apache.thrift.protocol.TBinaryProtocol; import org.apache.thrift.protocol.TField; import org.apache.thrift.protocol.TList; import org.apache.thrift.protocol.TMap; @@ -24,16 +26,24 @@ import org.apache.thrift.protocol.TProtocolUtil; import org.apache.thrift.protocol.TSet; import org.apache.thrift.protocol.TType; +import org.apache.thrift.transport.TMemoryInputTransport; +import org.apache.thrift.transport.TTransport; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; public class ThriftGenericRow implements TBase { private static final Logger log = Logger.get(ThriftGenericRow.class); private final Map values = new HashMap<>(); + private byte[] buf = null; + private int off = 0; + private int len = 0; public ThriftGenericRow() {} @@ -66,6 +76,23 @@ public String getFieldName() public void read(TProtocol iprot) throws TException { + TTransport trans = iprot.getTransport(); + buf = trans.getBuffer(); + off = trans.getBufferPosition(); + TProtocolUtil.skip(iprot, TType.STRUCT); + len = trans.getBufferPosition() - off; + } + + public void parse() throws TException + { + parse(null); + } + + public void parse(short[] thriftIds) throws TException + { + Set idSet = thriftIds == null ? null : new HashSet(Arrays.asList(ArrayUtils.toObject(thriftIds))); + TMemoryInputTransport trans = new TMemoryInputTransport(buf, off, len); + TBinaryProtocol iprot = new TBinaryProtocol(trans); TField field; iprot.readStructBegin(); while (true) { @@ -73,7 +100,12 @@ public void read(TProtocol iprot) throws TException if (field.type == TType.STOP) { break; } - values.put(field.id, readElem(iprot, field.type)); + if (idSet != null && !idSet.remove(Short.valueOf(field.id))) { + TProtocolUtil.skip(iprot, field.type); + } + else { + values.put(field.id, readElem(iprot, field.type)); + } iprot.readFieldEnd(); } iprot.readStructEnd(); @@ -115,6 +147,7 @@ private Object readStruct(TProtocol iprot) throws TException { ThriftGenericRow elem = new ThriftGenericRow(); elem.read(iprot); + elem.parse(); return elem; } diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java index 898b7a28ad1c..0293caf50be2 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java @@ -211,7 +211,7 @@ public boolean advanceNextPosition() Arrays.fill(loaded, false); // decode value - rowData = deserializer.deserialize(value); + rowData = deserializer.deserialize(value, thriftIds); return true; } From 00038dca1dbdfe414c904712f8d9d84ff77161e7 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 31 Mar 2017 19:21:41 -0700 Subject: [PATCH 158/331] Fix a mistake in splitting --- .../presto/twitter/hive/thrift/ThriftGeneralInputFormat.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java index 48c0f05bb0b3..5ecfdd4d7e57 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java @@ -130,7 +130,7 @@ public static InputSplit[] getLzoSplits( } long chunkEnd = index.alignSliceEndToIndex(chunkOffset + targetChunkSize, file.getLen()); - splits.add(new FileSplit(file.getPath(), chunkOffset, chunkEnd, job)); + splits.add(new FileSplit(file.getPath(), chunkOffset, chunkEnd - chunkOffset, job)); chunkOffset = chunkEnd; } From 3a9676b673a5e54f9e759c37bd629456bdeffcbd Mon Sep 17 00:00:00 2001 From: thomass Date: Mon, 10 Apr 2017 14:45:40 -0700 Subject: [PATCH 159/331] add table and columns info to QueryCompletedEvents --- twitter-eventlistener-plugin/pom.xml | 2 +- .../plugin/eventlistener/QueryCompletedEventScriber.java | 7 +++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 0ce3de21ca04..f404c698d7dd 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -35,7 +35,7 @@ com.twitter presto-thrift-java - 0.0.1 + 0.0.2 com.twitter diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java index e9c2faf3eda0..148968272af5 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java @@ -25,6 +25,10 @@ import io.airlift.log.Logger; import org.apache.thrift.TException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + /** * Class that scribes query completion events */ @@ -69,6 +73,9 @@ private static QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletedE thriftEvent.environment = eventContext.getEnvironment(); thriftEvent.catalog = eventContext.getCatalog().orElse(DASH); thriftEvent.schema = eventContext.getSchema().orElse(DASH); + Map> queriedColumnsByTable = new HashMap>(); + event.getIoMetadata().getInputs().forEach(input -> queriedColumnsByTable.put(input.getTable(), input.getColumns())); + thriftEvent.queried_columns_by_table = queriedColumnsByTable; thriftEvent.remote_client_address = eventContext.getRemoteClientAddress().orElse(DASH); thriftEvent.user_agent = eventContext.getUserAgent().orElse(DASH); thriftEvent.query_state = QueryState.valueOf(eventMetadata.getQueryState()); From 987c5a222041958a3ab88db7f2003b1300fa7029 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 13 Apr 2017 10:45:35 -0700 Subject: [PATCH 160/331] Better toString for HiveThriftFieldIdResolver; Allow EB config been override by table --- .../twitter/hive/thrift/HiveThriftFieldIdResolver.java | 6 +++++- .../twitter/hive/thrift/ThriftGeneralDeserializer.java | 1 - .../twitter/hive/thrift/ThriftGeneralInputFormat.java | 4 ++-- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java index f1ca6442a07b..6c6558d19ec1 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java @@ -19,6 +19,8 @@ import java.util.Properties; +import static com.google.common.base.MoreObjects.toStringHelper; + public class HiveThriftFieldIdResolver implements ThriftFieldIdResolver { @@ -74,6 +76,8 @@ public short getThriftId(int hiveIndex) @Override public String toString() { - return "HiveThriftFieldIdResolver with root:" + root; + return toStringHelper(this) + .add("root", root) + .toString(); } } diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralDeserializer.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralDeserializer.java index a2591fad8a62..5e131d84df51 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralDeserializer.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralDeserializer.java @@ -34,7 +34,6 @@ public void initialize(Configuration conf, Properties properties) String thriftClassName = properties.getProperty(SERIALIZATION_CLASS, null); checkCondition(thriftClassName != null, HIVE_INVALID_METADATA, "Table or partition is missing Hive deserializer property: %s", SERIALIZATION_CLASS); checkCondition(thriftClassName.equals(REQUIRED_SERIALIZATION_CLASS), HIVE_INVALID_METADATA, SERIALIZATION_CLASS + thriftClassName + " cannot match " + REQUIRED_SERIALIZATION_CLASS); - return; } public ThriftGenericRow deserialize(Writable writable, short[] thriftIds) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java index 5ecfdd4d7e57..b62a733b44cb 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java @@ -80,8 +80,8 @@ public RecordReader getRecordReader( Reporter reporter) throws IOException { - job.setBoolean("elephantbird.mapred.input.bad.record.check.only.in.close", false); - job.setFloat("elephantbird.mapred.input.bad.record.threshold", 0.0f); + job.setBooleanIfUnset("elephantbird.mapred.input.bad.record.check.only.in.close", false); + job.setIfUnset("elephantbird.mapred.input.bad.record.threshold", Float.toString(0.0f)); initialize((FileSplit) split, job); return super.getRecordReader(split, job, reporter); From 0a15a09a0ea9e91b4cf33ac8e2cd50ff0b1eec42 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 13 Apr 2017 19:19:05 -0700 Subject: [PATCH 161/331] Better Logging --- .../presto/hive/BackgroundHiveSplitLoader.java | 16 ++++++++++------ .../hive/thrift/ThriftGeneralDeserializer.java | 7 +++---- .../hive/thrift/ThriftGeneralInputFormat.java | 5 +++++ .../twitter/hive/thrift/ThriftGenericRow.java | 2 +- .../hive/thrift/ThriftHiveRecordCursor.java | 3 +-- 5 files changed, 20 insertions(+), 13 deletions(-) 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 1b68c2495423..5e22a027e60e 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 @@ -83,10 +83,14 @@ import static com.facebook.presto.hive.HiveUtil.isSplittable; import static com.facebook.presto.hive.metastore.MetastoreUtil.getHiveSchema; import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; +import static com.facebook.presto.twitter.hive.thrift.ThriftGeneralInputFormat.getLzoIndexPath; +import static com.facebook.presto.twitter.hive.thrift.ThriftGeneralInputFormat.getLzoSplits; +import static com.facebook.presto.twitter.hive.thrift.ThriftGeneralInputFormat.isLzoFile; import static com.google.common.base.Preconditions.checkState; import static java.lang.Math.toIntExact; import static java.lang.String.format; import static org.apache.hadoop.hive.common.FileUtils.HIDDEN_FILES_PATH_FILTER; +import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_NAME; public class BackgroundHiveSplitLoader implements HiveSplitLoader @@ -441,25 +445,25 @@ private void addLzoThriftSplitsToQueue( throws IOException { if (bucketHandle.isPresent()) { - throw new PrestoException(StandardErrorCode.NOT_SUPPORTED, "Bucketed table in ThriftGeneralInputFormat is not yet supported"); + throw new PrestoException(StandardErrorCode.NOT_SUPPORTED, format("Bucketed table %s in ThriftGeneralInputFormat is not yet supported", schema.getProperty(META_TABLE_NAME))); } List> iteratorList = new ArrayList<>(); while (hiveFileIterator.hasNext()) { LocatedFileStatus file = hiveFileIterator.next(); - if (!ThriftGeneralInputFormat.lzoSuffixFilter.accept(file.getPath())) { + if (!isLzoFile(file.getPath())) { continue; } Configuration targetConfiguration = hdfsEnvironment.getConfiguration(file.getPath()); JobConf targetJob = new JobConf(targetConfiguration); - FileSystem indexFilesystem = hdfsEnvironment.getFileSystem(session.getUser(), ThriftGeneralInputFormat.getLzoIndexPath(file.getPath())); - InputSplit[] targetSplits = ThriftGeneralInputFormat.getLzoSplits(targetJob, file, indexFilesystem, remainingInitialSplits, maxInitialSplitSize, maxSplitSize); + FileSystem indexFilesystem = hdfsEnvironment.getFileSystem(session.getUser(), getLzoIndexPath(file.getPath())); + InputSplit[] targetSplits = getLzoSplits(targetJob, file, indexFilesystem, remainingInitialSplits, maxInitialSplitSize, maxSplitSize); log.debug("For file at %s, get number of splits %s", file.getPath(), targetSplits.length); + FileSystem targetFilesystem = hdfsEnvironment.getFileSystem(session.getUser(), file.getPath()); + FileStatus targetFile = targetFilesystem.getFileStatus(file.getPath()); for (InputSplit inputSplit : targetSplits) { FileSplit split = (FileSplit) inputSplit; - FileSystem targetFilesystem = hdfsEnvironment.getFileSystem(session.getUser(), split.getPath()); - FileStatus targetFile = targetFilesystem.getFileStatus(split.getPath()); iteratorList.add(createHiveSplitIterator( partitionName, targetFile.getPath().toString(), diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralDeserializer.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralDeserializer.java index 5e131d84df51..001994f7a7e4 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralDeserializer.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralDeserializer.java @@ -27,9 +27,8 @@ public class ThriftGeneralDeserializer { - private static final String REQUIRED_SERIALIZATION_CLASS = "com.facebook.presto.twitter.hive.thrift.ThriftGenericRow"; - - public void initialize(Configuration conf, Properties properties) + private static final String REQUIRED_SERIALIZATION_CLASS = ThriftGenericRow.class.getName(); + public ThriftGeneralDeserializer(Configuration conf, Properties properties) { String thriftClassName = properties.getProperty(SERIALIZATION_CLASS, null); checkCondition(thriftClassName != null, HIVE_INVALID_METADATA, "Table or partition is missing Hive deserializer property: %s", SERIALIZATION_CLASS); @@ -44,7 +43,7 @@ public ThriftGenericRow deserialize(Writable writable, short[] thriftIds) row.parse(thriftIds); } catch (TException e) { - throw new IllegalStateException("Generic row failed to parse values", e); + throw new IllegalStateException("ThriftGenericRow failed to parse values", e); } return row; } diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java index b62a733b44cb..ee358096897d 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java @@ -87,6 +87,11 @@ public RecordReader getRecordReader( return super.getRecordReader(split, job, reporter); } + public static boolean isLzoFile(Path filePath) + { + return lzoSuffixFilter.accept(filePath); + } + public static Path getLzoIndexPath(Path lzoPath) { return lzoPath.suffix(LzoIndex.LZO_INDEX_SUFFIX); diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java index 2d911b09061f..5f33715fe330 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java @@ -139,8 +139,8 @@ private Object readElem(TProtocol iprot, byte type) throws TException return readMap(iprot); default: TProtocolUtil.skip(iprot, type); + return null; } - return null; } private Object readStruct(TProtocol iprot) throws TException diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java index 0293caf50be2..ffbe348950db 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java @@ -131,8 +131,7 @@ public ThriftHiveRecordCursor( this.hiveStorageTimeZone = hiveStorageTimeZone; this.thriftFieldIdResolver = thriftFieldIdResolver; - this.deserializer = new ThriftGeneralDeserializer(); - deserializer.initialize(new Configuration(false), splitSchema); + this.deserializer = new ThriftGeneralDeserializer(new Configuration(false), splitSchema); int size = columns.size(); From 09de6259c60874f4aa4479aaf859717c17746ca4 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 14 Apr 2017 20:40:58 -0700 Subject: [PATCH 162/331] Refactor thrift field id resolver --- .../presto/hive/HiveClientModule.java | 6 +- .../thrift/HiveThriftFieldIdResolver.java | 67 ++++++++++--------- .../HiveThriftFieldIdResolverFactory.java | 44 ++++++++++++ .../hive/thrift/ThriftFieldIdResolver.java | 3 - .../thrift/ThriftFieldIdResolverFactory.java | 21 ++++++ .../ThriftHiveRecordCursorProvider.java | 8 +-- 6 files changed, 106 insertions(+), 43 deletions(-) create mode 100644 presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolverFactory.java create mode 100644 presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftFieldIdResolverFactory.java 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 9c12f947726a..d4f76e32ad3d 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 @@ -27,8 +27,8 @@ import com.facebook.presto.spi.connector.ConnectorSplitManager; import com.facebook.presto.spi.type.TypeManager; import com.facebook.presto.twitter.hive.PooledHiveMetastoreClientFactory; -import com.facebook.presto.twitter.hive.thrift.HiveThriftFieldIdResolver; -import com.facebook.presto.twitter.hive.thrift.ThriftFieldIdResolver; +import com.facebook.presto.twitter.hive.thrift.HiveThriftFieldIdResolverFactory; +import com.facebook.presto.twitter.hive.thrift.ThriftFieldIdResolverFactory; import com.facebook.presto.twitter.hive.thrift.ThriftHiveRecordCursorProvider; import com.google.inject.Binder; import com.google.inject.Module; @@ -91,7 +91,7 @@ public void configure(Binder binder) binder.bind(NodeManager.class).toInstance(nodeManager); binder.bind(TypeManager.class).toInstance(typeManager); binder.bind(PageIndexerFactory.class).toInstance(pageIndexerFactory); - binder.bind(ThriftFieldIdResolver.class).toInstance(new HiveThriftFieldIdResolver()); + binder.bind(ThriftFieldIdResolverFactory.class).toInstance(new HiveThriftFieldIdResolverFactory()); Multibinder recordCursorProviderBinder = Multibinder.newSetBinder(binder, HiveRecordCursorProvider.class); recordCursorProviderBinder.addBinding().to(ParquetRecordCursorProvider.class).in(Scopes.SINGLETON); diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java index 6c6558d19ec1..edc3ce219faa 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java @@ -14,62 +14,62 @@ package com.facebook.presto.twitter.hive.thrift; import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.ObjectMapper; -import io.airlift.log.Logger; -import java.util.Properties; +import java.util.HashMap; +import java.util.Map; +import static com.facebook.presto.hive.HiveErrorCode.HIVE_INVALID_METADATA; +import static com.facebook.presto.hive.HiveUtil.checkCondition; import static com.google.common.base.MoreObjects.toStringHelper; public class HiveThriftFieldIdResolver implements ThriftFieldIdResolver { - private static final Logger log = Logger.get(HiveThriftFieldIdResolver.class); - private static final ObjectMapper objectMapper = new ObjectMapper(); - public static final String THRIFT_FIELD_ID_JSON = "thrift.field.id.json"; private final JsonNode root; - - public HiveThriftFieldIdResolver() - { - this.root = null; - } + private final Map nestedResolvers = new HashMap<>(); + private final Map thriftIds = new HashMap<>(); public HiveThriftFieldIdResolver(JsonNode root) { this.root = root; } - public ThriftFieldIdResolver initialize(Properties schema) + public short getThriftId(int hiveIndex) { - String jsonData = schema.getProperty(THRIFT_FIELD_ID_JSON); - try { - return new HiveThriftFieldIdResolver(objectMapper.readTree(jsonData)); - } - catch (Exception e) { - log.debug("Got an exception %s in initialize, schema: %s", e.getMessage(), schema); - return new HiveThriftFieldIdResolver(); + if (root == null) { + return (short) (hiveIndex + 1); } - } - public ThriftFieldIdResolver getNestedResolver(int hiveIndex) - { - try { - return new HiveThriftFieldIdResolver(root.get(String.valueOf(hiveIndex))); + Short thriftId = thriftIds.get(Integer.valueOf(hiveIndex)); + if (thriftId != null) { + return thriftId.shortValue(); } - catch (Exception e) { - log.debug("Got an exception %s in getNestedResolver, root: %s, want the hiveIndex: %s", e.getMessage(), root, hiveIndex); - return new HiveThriftFieldIdResolver(); + else { + JsonNode child = root.get(String.valueOf(hiveIndex)); + checkCondition(child != null, HIVE_INVALID_METADATA, "Missed json value for hiveIndex: %s, root: %s", hiveIndex, root); + checkCondition(child.get("id") != null, HIVE_INVALID_METADATA, "Missed key id for hiveIndex: %s, root: %s", hiveIndex, root); + thriftId = Short.valueOf((short) child.get("id").asInt()); + thriftIds.put(Integer.valueOf(hiveIndex), thriftId); + return thriftId; } } - public short getThriftId(int hiveIndex) + public ThriftFieldIdResolver getNestedResolver(int hiveIndex) { - try { - return (short) root.get(String.valueOf(hiveIndex)).get("id").asInt(); + ThriftFieldIdResolver nestedResolver = nestedResolvers.get(Integer.valueOf(hiveIndex)); + if (nestedResolver != null) { + return nestedResolver; } - catch (Exception e) { - log.debug("Got an exception %s in getThriftId, root: %s, want the hiveIndex: %s", e.getMessage(), root, hiveIndex); - return (short) (hiveIndex + 1); + else { + JsonNode child = null; + if (root != null) { + child = root.get(String.valueOf(hiveIndex)); + } + // what if the child == null? + // checkCondition(child != null, HIVE_INVALID_METADATA, "Missed json value for hiveIndex: %s, root: %s", hiveIndex, root); + nestedResolver = new HiveThriftFieldIdResolver(child); + nestedResolvers.put(Integer.valueOf(hiveIndex), nestedResolver); + return nestedResolver; } } @@ -78,6 +78,7 @@ public String toString() { return toStringHelper(this) .add("root", root) + .add("nestedResolvers", nestedResolvers) .toString(); } } diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolverFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolverFactory.java new file mode 100644 index 000000000000..1df65fa63796 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolverFactory.java @@ -0,0 +1,44 @@ +/* + * 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.twitter.hive.thrift; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.airlift.log.Logger; + +import java.io.IOException; +import java.util.Properties; + +public class HiveThriftFieldIdResolverFactory + implements ThriftFieldIdResolverFactory +{ + private static final Logger log = Logger.get(HiveThriftFieldIdResolverFactory.class); + private static final ObjectMapper objectMapper = new ObjectMapper(); + public static final String THRIFT_FIELD_ID_JSON = "thrift.field.id.json"; + + public ThriftFieldIdResolver createResolver(Properties schema) + { + JsonNode root = null; + String jsonData = schema.getProperty(THRIFT_FIELD_ID_JSON); + if (jsonData != null) { + try { + root = objectMapper.readTree(jsonData); + } + catch (IOException e) { + log.debug(e, "Failed to createResolver, schema: %s", schema); + } + } + return new HiveThriftFieldIdResolver(root); + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftFieldIdResolver.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftFieldIdResolver.java index 068e485e8989..83047e1d953f 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftFieldIdResolver.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftFieldIdResolver.java @@ -13,11 +13,8 @@ */ package com.facebook.presto.twitter.hive.thrift; -import java.util.Properties; - public interface ThriftFieldIdResolver { - ThriftFieldIdResolver initialize(Properties schema); ThriftFieldIdResolver getNestedResolver(int hiveIndex); short getThriftId(int hiveIndex); } diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftFieldIdResolverFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftFieldIdResolverFactory.java new file mode 100644 index 000000000000..034308aaa569 --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftFieldIdResolverFactory.java @@ -0,0 +1,21 @@ +/* + * 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.twitter.hive.thrift; + +import java.util.Properties; + +public interface ThriftFieldIdResolverFactory +{ + ThriftFieldIdResolver createResolver(Properties schema); +} 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 841927816b6b..2e3d7f0f6cf6 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 @@ -46,13 +46,13 @@ public class ThriftHiveRecordCursorProvider .add("com.facebook.presto.twitter.hive.thrift.ThriftGeneralDeserializer") .build(); private final HdfsEnvironment hdfsEnvironment; - private final ThriftFieldIdResolver thriftFieldIdResolver; + private final ThriftFieldIdResolverFactory thriftFieldIdResolverFactory; @Inject - public ThriftHiveRecordCursorProvider(HdfsEnvironment hdfsEnvironment, ThriftFieldIdResolver thriftFieldIdResolver) + public ThriftHiveRecordCursorProvider(HdfsEnvironment hdfsEnvironment, ThriftFieldIdResolverFactory thriftFieldIdResolverFactory) { this.hdfsEnvironment = requireNonNull(hdfsEnvironment, "hdfsEnvironment is null"); - this.thriftFieldIdResolver = requireNonNull(thriftFieldIdResolver, "thriftFieldIdResolver is null"); + this.thriftFieldIdResolverFactory = requireNonNull(thriftFieldIdResolverFactory, "thriftFieldIdResolverFactory is null"); } @Override @@ -83,7 +83,7 @@ public Optional createRecordCursor( columns, hiveStorageTimeZone, typeManager, - thriftFieldIdResolver.initialize(schema))); + thriftFieldIdResolverFactory.createResolver(schema))); } @SuppressWarnings("unchecked") From 6d4eaafe3870a9a94415e738b38b64f6cba02218 Mon Sep 17 00:00:00 2001 From: thomass Date: Sun, 16 Apr 2017 13:46:56 -0700 Subject: [PATCH 163/331] mave tag to 0.173 first --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 47 files changed, 49 insertions(+), 49 deletions(-) diff --git a/pom.xml b/pom.xml index 0c4d8d63a228..e0503b56c4ea 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.170-tw-0.32 + 0.173 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 30fe8b9165f3..6b8a0ea5c1e9 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index d7f7024682e3..148f63c1c244 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 31680c745445..43c22888b4ec 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index ecf712772755..e149d1896e6a 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 362b1f8f3241..ad92ffb73fcf 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index e922fcbb916a..30db245cb4bb 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.170-tw-0.32 + 0.173 presto-benchmark diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index c2b8f0c378eb..bc75462e53ba 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 9b0d4f9f79d8..e3c9f43dfef1 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 289cad7109fa..5cc5401e2662 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 4e1718d8c8a4..001dbd4748d5 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 2f5114c3a4a7..fe07db0f630e 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 61521de2c57c..dd91312b61d4 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 9a6a0724f65f..a184c81691e5 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index e31ee043a73b..d3cc8ad84974 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 2e01b0fd9bcd..f7072fd6c973 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index b18f5fd4d701..11598d13a3aa 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 4cdfbe145f3d..f3dc07fd0d26 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index afc7eccb483a..3157e6289b2b 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 443e93bfa2bb..ebac6c7cac8d 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 7dc9f4e215ad..93b93454a96c 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 22b22b95232f..441df3abbd17 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index ceecc3da62a3..5505d7ad0506 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 1e4125da4f11..e21530bd3f7d 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-main diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 2a634850be7f..120ef8203efa 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index ac06d2ea198f..fd182dfb964a 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 062ee8c05e8f..e9325bb1a53c 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 2a9879c22f1c..154308c7a3b1 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 88d5a99cfe79..0e6aaf43dfe9 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 8c6fae50a5a7..481b34a63f5f 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index ddf520bebb66..ff76ac8cb3fb 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 1dba94f35a14..d2d08db8b05a 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 2c837b379e1d..46192eeca751 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.170-tw-0.32 + 0.173 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 4f83793ca631..d4ad9695363a 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 00f49ad4d0db..a4abf7e454b4 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 6db7c2373423..32484a240a34 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 974ffffabd0a..6887cee87173 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index a6a641bdab23..f16790697439 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index a2b420c7465c..765195387829 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 5c76a7a23ba2..fea50eb030c6 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index b2853ac43c50..586912612e54 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index ef8c6726398d..a4397c2fc0c0 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 0d058bb07f73..a3d1b570fc25 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 0420d4fe046a..150f230d0585 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.170-tw-0.32 + 0.173 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index dfd00558a2a5..45137f24398f 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index f3d599d677f4..82a5489f6d31 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index f404c698d7dd..7f8b2c7dc6b3 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.170-tw-0.32 + 0.173 provided From becae616f29b824fb056342b640b1910d5d8248b Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Mon, 17 Apr 2017 15:16:56 -0700 Subject: [PATCH 164/331] Support deserialization when the SerDe is set as LazyBinarySerDe --- .../java/com/facebook/presto/hive/HiveUtil.java | 13 +++++++++++++ .../hive/thrift/HiveThriftFieldIdResolver.java | 6 +++--- .../hive/thrift/ThriftHiveRecordCursorProvider.java | 10 ++++++++++ 3 files changed, 26 insertions(+), 3 deletions(-) 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 dc217a6b6e5c..a1a2ea79dd96 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 @@ -27,6 +27,7 @@ import com.facebook.presto.spi.type.StandardTypes; import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.VarcharType; +import com.facebook.presto.twitter.hive.thrift.ThriftGeneralInputFormat; import com.google.common.base.Joiner; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; @@ -118,6 +119,7 @@ import static java.util.Objects.requireNonNull; import static org.apache.hadoop.hive.common.FileUtils.unescapePathName; import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.FILE_INPUT_FORMAT; +import static org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS; import static org.apache.hadoop.hive.serde.serdeConstants.DECIMAL_TYPE_NAME; import static org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_LIB; import static org.apache.hadoop.hive.serde2.ColumnProjectionUtils.READ_ALL_COLUMNS; @@ -226,6 +228,10 @@ public static void setReadColumns(Configuration configuration, List rea return MapredParquetInputFormat.class; } + if ("com.twitter.elephantbird.mapred.input.HiveMultiInputFormat".equals(inputFormatName)) { + return ThriftGeneralInputFormat.class; + } + Class clazz = conf.getClassByName(inputFormatName); // TODO: remove redundant cast to Object after IDEA-118533 is fixed return (Class>) (Object) clazz.asSubclass(InputFormat.class); @@ -238,6 +244,13 @@ static String getInputFormatName(Properties schema) return name; } + public static String getSerializationClassName(Properties schema) + { + String name = schema.getProperty(SERIALIZATION_CLASS); + checkCondition(name != null, HIVE_INVALID_METADATA, "Table or partition is missing Hive property: %s", SERIALIZATION_CLASS); + return name; + } + public static long parseHiveDate(String value) { long millis = HIVE_DATE_PARSER.parseMillis(value); diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java index edc3ce219faa..d187bea64297 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java @@ -40,7 +40,7 @@ public short getThriftId(int hiveIndex) return (short) (hiveIndex + 1); } - Short thriftId = thriftIds.get(Integer.valueOf(hiveIndex)); + Short thriftId = thriftIds.get(hiveIndex); if (thriftId != null) { return thriftId.shortValue(); } @@ -56,7 +56,7 @@ public short getThriftId(int hiveIndex) public ThriftFieldIdResolver getNestedResolver(int hiveIndex) { - ThriftFieldIdResolver nestedResolver = nestedResolvers.get(Integer.valueOf(hiveIndex)); + ThriftFieldIdResolver nestedResolver = nestedResolvers.get(hiveIndex); if (nestedResolver != null) { return nestedResolver; } @@ -68,7 +68,7 @@ public ThriftFieldIdResolver getNestedResolver(int hiveIndex) // what if the child == null? // checkCondition(child != null, HIVE_INVALID_METADATA, "Missed json value for hiveIndex: %s, root: %s", hiveIndex, root); nestedResolver = new HiveThriftFieldIdResolver(child); - nestedResolvers.put(Integer.valueOf(hiveIndex), nestedResolver); + nestedResolvers.put(hiveIndex, nestedResolver); return nestedResolver; } } 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 2e3d7f0f6cf6..e569f60e0cd8 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 @@ -36,14 +36,18 @@ import static com.facebook.presto.hive.HiveUtil.createRecordReader; import static com.facebook.presto.hive.HiveUtil.getDeserializerClassName; +import static com.facebook.presto.hive.HiveUtil.getSerializationClassName; import static java.util.Objects.requireNonNull; public class ThriftHiveRecordCursorProvider implements HiveRecordCursorProvider { + private static final String LAZY_BINARY_SERDE = "org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe"; + private static final String THRIFT_GENERIC_ROW = ThriftGenericRow.class.getName(); private static final Set THRIFT_SERDE_CLASS_NAMES = ImmutableSet.builder() .add("com.facebook.presto.twitter.hive.thrift.ThriftGeneralSerDe") .add("com.facebook.presto.twitter.hive.thrift.ThriftGeneralDeserializer") + .add(LAZY_BINARY_SERDE) .build(); private final HdfsEnvironment hdfsEnvironment; private final ThriftFieldIdResolverFactory thriftFieldIdResolverFactory; @@ -73,6 +77,12 @@ public Optional createRecordCursor( return Optional.empty(); } + // We only allow the table which specified its serialization class is compatible to + // our thrift general row, if the SerDe is LazyBinarySerDe. + if (LAZY_BINARY_SERDE.equals(getDeserializerClassName(schema)) && !THRIFT_GENERIC_ROW.equals(getSerializationClassName(schema))) { + return Optional.empty(); + } + RecordReader recordReader = hdfsEnvironment.doAs(session.getUser(), () -> createRecordReader(configuration, path, start, length, schema, columns)); From cce50236afa7bbf417bbe4aab690163ca2bfc2f4 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Mon, 17 Apr 2017 16:19:17 -0700 Subject: [PATCH 165/331] Move functions out of ThriftGeneralInputFormat --- .../hive/BackgroundHiveSplitLoader.java | 52 ++++++++++--- .../com/facebook/presto/hive/HiveUtil.java | 14 ++++ .../hive/thrift/ThriftGeneralInputFormat.java | 77 +------------------ 3 files changed, 60 insertions(+), 83 deletions(-) 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 5e22a027e60e..ff5916b0631f 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 @@ -32,6 +32,7 @@ import com.google.common.collect.Iterators; import com.google.common.collect.PeekingIterator; import com.google.common.io.CharStreams; +import com.hadoop.compression.lzo.LzoIndex; import io.airlift.log.Logger; import io.airlift.units.DataSize; import org.apache.hadoop.conf.Configuration; @@ -80,12 +81,10 @@ import static com.facebook.presto.hive.HiveSessionProperties.getMaxSplitSize; import static com.facebook.presto.hive.HiveUtil.checkCondition; import static com.facebook.presto.hive.HiveUtil.getInputFormat; +import static com.facebook.presto.hive.HiveUtil.isLzoCompressedFile; import static com.facebook.presto.hive.HiveUtil.isSplittable; import static com.facebook.presto.hive.metastore.MetastoreUtil.getHiveSchema; import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; -import static com.facebook.presto.twitter.hive.thrift.ThriftGeneralInputFormat.getLzoIndexPath; -import static com.facebook.presto.twitter.hive.thrift.ThriftGeneralInputFormat.getLzoSplits; -import static com.facebook.presto.twitter.hive.thrift.ThriftGeneralInputFormat.isLzoFile; import static com.google.common.base.Preconditions.checkState; import static java.lang.Math.toIntExact; import static java.lang.String.format; @@ -451,14 +450,10 @@ private void addLzoThriftSplitsToQueue( while (hiveFileIterator.hasNext()) { LocatedFileStatus file = hiveFileIterator.next(); - if (!isLzoFile(file.getPath())) { + if (!isLzoCompressedFile(file.getPath())) { continue; } - - Configuration targetConfiguration = hdfsEnvironment.getConfiguration(file.getPath()); - JobConf targetJob = new JobConf(targetConfiguration); - FileSystem indexFilesystem = hdfsEnvironment.getFileSystem(session.getUser(), getLzoIndexPath(file.getPath())); - InputSplit[] targetSplits = getLzoSplits(targetJob, file, indexFilesystem, remainingInitialSplits, maxInitialSplitSize, maxSplitSize); + InputSplit[] targetSplits = createLzoSplits(file); log.debug("For file at %s, get number of splits %s", file.getPath(), targetSplits.length); FileSystem targetFilesystem = hdfsEnvironment.getFileSystem(session.getUser(), file.getPath()); FileStatus targetFile = targetFilesystem.getFileStatus(file.getPath()); @@ -486,6 +481,40 @@ private void addLzoThriftSplitsToQueue( addToHiveSplitSourceRoundRobin(iteratorList); } + private InputSplit[] createLzoSplits(LocatedFileStatus file) + throws IOException + { + Configuration targetConfiguration = hdfsEnvironment.getConfiguration(file.getPath()); + JobConf job = new JobConf(targetConfiguration); + FileSystem indexFilesystem = hdfsEnvironment.getFileSystem(session.getUser(), getLzoIndexPath(file.getPath())); + LzoIndex index = LzoIndex.readIndex(indexFilesystem, file.getPath()); + if (index.isEmpty()) { + InputSplit[] splits = new InputSplit[1]; + splits[0] = new FileSplit(file.getPath(), 0, file.getLen(), job); + return splits; + } + + List splits = new ArrayList<>(); + long chunkOffset = 0; + while (chunkOffset < file.getLen()) { + long targetChunkSize; + if (remainingInitialSplits.decrementAndGet() >= 0) { + targetChunkSize = maxInitialSplitSize.toBytes(); + } + else { + long maxBytes = maxSplitSize.toBytes(); + int chunks = toIntExact((long) Math.ceil((file.getLen() - chunkOffset) * 1.0 / maxBytes)); + targetChunkSize = (long) Math.ceil((file.getLen() - chunkOffset) * 1.0 / chunks); + } + long chunkEnd = index.alignSliceEndToIndex(chunkOffset + targetChunkSize, file.getLen()); + + splits.add(new FileSplit(file.getPath(), chunkOffset, chunkEnd - chunkOffset, job)); + chunkOffset = chunkEnd; + } + + return splits.toArray(new InputSplit[0]); + } + private boolean addSplitsToSource( InputSplit[] targetSplits, String partitionName, @@ -742,4 +771,9 @@ private static String getPartitionLocation(Table table, Optional part } return partition.get().getStorage().getLocation(); } + + private static Path getLzoIndexPath(Path lzoPath) + { + return lzoPath.suffix(LzoIndex.LZO_INDEX_SUFFIX); + } } 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 a1a2ea79dd96..1db6cad7d545 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 @@ -37,6 +37,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hive.common.JavaUtils; import org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat; import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat; @@ -142,6 +143,14 @@ public final class HiveUtil private static final String BIG_DECIMAL_POSTFIX = "BD"; + private static final PathFilter LZOP_DEFAULT_SUFFIX_FILTER = new PathFilter() { + @Override + public boolean accept(Path path) + { + return path.toString().endsWith(".lzo"); + } + }; + static { DateTimeParser[] timestampWithoutTimeZoneParser = { DateTimeFormat.forPattern("yyyy-M-d").getParser(), @@ -296,6 +305,11 @@ static boolean isSplittable(InputFormat inputFormat, FileSystem fileSystem } } + static boolean isLzoCompressedFile(Path filePath) + { + return LZOP_DEFAULT_SUFFIX_FILTER.accept(filePath); + } + public static StructObjectInspector getTableObjectInspector(Properties schema) { return getTableObjectInspector(getDeserializer(schema)); diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java index ee358096897d..ae0901d65c65 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java @@ -13,16 +13,10 @@ */ package com.facebook.presto.twitter.hive.thrift; -import com.hadoop.compression.lzo.LzoIndex; import com.twitter.elephantbird.mapred.input.DeprecatedFileInputFormatWrapper; import com.twitter.elephantbird.mapreduce.input.MultiInputFormat; import com.twitter.elephantbird.mapreduce.io.BinaryWritable; import com.twitter.elephantbird.util.TypeRef; -import io.airlift.units.DataSize; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.LocatedFileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.mapred.InputSplit; @@ -31,29 +25,19 @@ import org.apache.hadoop.mapred.Reporter; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.atomic.AtomicInteger; import static com.facebook.presto.hive.HiveErrorCode.HIVE_INVALID_METADATA; import static com.facebook.presto.hive.HiveUtil.checkCondition; -import static java.lang.Math.toIntExact; import static org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS; /** - * Customized version of com.twitter.elephantbird.mapred.input.HiveMultiInputFormat + * Mirror of com.twitter.elephantbird.mapred.input.HiveMultiInputFormat allows to pass the thriftClassName + * directly as a property of JobConfig. + * PR for twitter/elephant-bird: https://github.com/twitter/elephant-bird/pull/481 */ @SuppressWarnings("deprecation") public class ThriftGeneralInputFormat extends DeprecatedFileInputFormatWrapper { - public static final PathFilter lzoSuffixFilter = new PathFilter() { - @Override - public boolean accept(Path path) - { - return path.toString().endsWith(".lzo"); - } - }; - public ThriftGeneralInputFormat() { super(new MultiInputFormat()); @@ -86,59 +70,4 @@ public RecordReader getRecordReader( initialize((FileSplit) split, job); return super.getRecordReader(split, job, reporter); } - - public static boolean isLzoFile(Path filePath) - { - return lzoSuffixFilter.accept(filePath); - } - - public static Path getLzoIndexPath(Path lzoPath) - { - return lzoPath.suffix(LzoIndex.LZO_INDEX_SUFFIX); - } - - public static InputSplit[] getLzoSplits( - JobConf job, - LocatedFileStatus file) - throws IOException - { - InputSplit[] splits = new InputSplit[1]; - splits[0] = new FileSplit(file.getPath(), 0, file.getLen(), job); - return splits; - } - - public static InputSplit[] getLzoSplits( - JobConf job, - LocatedFileStatus file, - FileSystem indexFilesystem, - AtomicInteger remainingInitialSplits, - DataSize maxInitialSplitSize, - DataSize maxSplitSize) - throws IOException - { - LzoIndex index = LzoIndex.readIndex(indexFilesystem, file.getPath()); - if (index.isEmpty()) { - return getLzoSplits(job, file); - } - - List splits = new ArrayList<>(); - long chunkOffset = 0; - while (chunkOffset < file.getLen()) { - long targetChunkSize; - if (remainingInitialSplits.decrementAndGet() >= 0) { - targetChunkSize = maxInitialSplitSize.toBytes(); - } - else { - long maxBytes = maxSplitSize.toBytes(); - int chunks = toIntExact((long) Math.ceil((file.getLen() - chunkOffset) * 1.0 / maxBytes)); - targetChunkSize = (long) Math.ceil((file.getLen() - chunkOffset) * 1.0 / chunks); - } - long chunkEnd = index.alignSliceEndToIndex(chunkOffset + targetChunkSize, file.getLen()); - - splits.add(new FileSplit(file.getPath(), chunkOffset, chunkEnd - chunkOffset, job)); - chunkOffset = chunkEnd; - } - - return splits.toArray(new InputSplit[0]); - } } From 359479ccc42c6c80827919adcf987ffc5973340c Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 18 Apr 2017 13:59:53 -0700 Subject: [PATCH 166/331] move elephantbird config setting into cursor provider --- .../main/java/com/facebook/presto/hive/HiveUtil.java | 2 +- .../twitter/hive/thrift/ThriftGeneralInputFormat.java | 3 --- .../hive/thrift/ThriftHiveRecordCursorProvider.java | 10 ++++++++++ 3 files changed, 11 insertions(+), 4 deletions(-) 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 1db6cad7d545..e4d401aac30f 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 @@ -183,7 +183,7 @@ private HiveUtil() // propagate serialization configuration to getRecordReader schema.stringPropertyNames().stream() - .filter(name -> name.startsWith("serialization.")) + .filter(name -> name.startsWith("serialization.") || name.startsWith("elephantbird.")) .forEach(name -> jobConf.set(name, schema.getProperty(name))); try { diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java index ae0901d65c65..13c78d85fad0 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java @@ -64,9 +64,6 @@ public RecordReader getRecordReader( Reporter reporter) throws IOException { - job.setBooleanIfUnset("elephantbird.mapred.input.bad.record.check.only.in.close", false); - job.setIfUnset("elephantbird.mapred.input.bad.record.threshold", Float.toString(0.0f)); - initialize((FileSplit) split, job); return super.getRecordReader(split, job, reporter); } 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 e569f60e0cd8..10439e2b9d53 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 @@ -83,6 +83,9 @@ public Optional createRecordCursor( return Optional.empty(); } + setPropertyIfUnset(schema, "elephantbird.mapred.input.bad.record.check.only.in.close", Boolean.toString(false)); + setPropertyIfUnset(schema, "elephantbird.mapred.input.bad.record.threshold", Float.toString(0.0f)); + RecordReader recordReader = hdfsEnvironment.doAs(session.getUser(), () -> createRecordReader(configuration, path, start, length, schema, columns)); @@ -101,4 +104,11 @@ public Optional createRecordCursor( { return (RecordReader) recordReader; } + + private static void setPropertyIfUnset(Properties schema, String key, String value) + { + if (schema.getProperty(key) == null) { + schema.setProperty(key, value); + } + } } From 7bb29941e0c5e597af00dc6aa443c0b9006dd856 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 19 Apr 2017 18:23:55 -0700 Subject: [PATCH 167/331] Initial commit to support LZO Thrift write --- .../presto/hive/HiveCompressionCodec.java | 4 +- .../presto/hive/HiveStorageFormat.java | 7 ++ .../twitter/hive/thrift/ThriftGenericRow.java | 83 ++++++++++++++++++- .../presto/hive/TestHiveFileFormats.java | 56 +++++++++++++ 4 files changed, 146 insertions(+), 4 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveCompressionCodec.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveCompressionCodec.java index 34cea5b9d39d..de314b5fdbfc 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveCompressionCodec.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveCompressionCodec.java @@ -16,6 +16,7 @@ import org.apache.hadoop.hive.ql.io.orc.CompressionKind; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.GzipCodec; +import org.apache.hadoop.io.compress.LzoCodec; import org.apache.hadoop.io.compress.SnappyCodec; import parquet.hadoop.metadata.CompressionCodecName; @@ -27,7 +28,8 @@ public enum HiveCompressionCodec { NONE(null, CompressionKind.NONE, CompressionCodecName.UNCOMPRESSED), SNAPPY(SnappyCodec.class, CompressionKind.SNAPPY, CompressionCodecName.SNAPPY), - GZIP(GzipCodec.class, CompressionKind.ZLIB, CompressionCodecName.GZIP); + GZIP(GzipCodec.class, CompressionKind.ZLIB, CompressionCodecName.GZIP), + LZO(LzoCodec.class, CompressionKind.LZO, CompressionCodecName.LZO); private final Optional> codec; private final CompressionKind orcCompressionKind; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveStorageFormat.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveStorageFormat.java index 2d481c7bcbe5..fe4c3b73cbc0 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveStorageFormat.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveStorageFormat.java @@ -14,6 +14,7 @@ package com.facebook.presto.hive; import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.twitter.hive.thrift.ThriftGeneralInputFormat; import io.airlift.units.DataSize; import io.airlift.units.DataSize.Unit; import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat; @@ -32,6 +33,7 @@ import org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe; import org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe; import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe; +import org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory; import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; @@ -91,6 +93,11 @@ public enum HiveStorageFormat TEXTFILE(LazySimpleSerDe.class.getName(), TextInputFormat.class.getName(), HiveIgnoreKeyTextOutputFormat.class.getName(), + new DataSize(8, Unit.MEGABYTE)), + THRIFTBINARY( + LazyBinarySerDe.class.getName(), + ThriftGeneralInputFormat.class.getName(), + HiveIgnoreKeyTextOutputFormat.class.getName(), new DataSize(8, Unit.MEGABYTE)); private final String serde; diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java index 5f33715fe330..559fe7825756 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java @@ -25,6 +25,7 @@ import org.apache.thrift.protocol.TProtocol; import org.apache.thrift.protocol.TProtocolUtil; import org.apache.thrift.protocol.TSet; +import org.apache.thrift.protocol.TStruct; import org.apache.thrift.protocol.TType; import org.apache.thrift.transport.TMemoryInputTransport; import org.apache.thrift.transport.TTransport; @@ -213,12 +214,88 @@ public boolean isSet(Fields field) public void setFieldValue(Fields field, Object value) { - throw new UnsupportedOperationException("ThriftGenericRow.setFieldValue is not supported."); + values.put(field.getThriftFieldId(), value); } - public void write(TProtocol oprot) + public void write(TProtocol oprot) throws TException { - throw new UnsupportedOperationException("ThriftGenericRow.write is not supported."); + oprot.writeStructBegin(new TStruct("dummy")); + values.entrySet().stream() + .filter(v -> v.getValue() != null) + .forEach(v -> writeField(v, oprot)); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + private void writeField(Map.Entry field, TProtocol oprot) + { + Object value = field.getValue(); + short thriftId = field.getKey(); + try { + if (value instanceof Boolean) { + oprot.writeFieldBegin(new TField("dummy", TType.BOOL, thriftId)); + oprot.writeBool((Boolean) value); + } + else if (value instanceof Byte) { + oprot.writeFieldBegin(new TField("dummy", TType.BYTE, thriftId)); + oprot.writeByte((Byte) value); + } + else if (value instanceof Short) { + oprot.writeFieldBegin(new TField("dummy", TType.I16, thriftId)); + oprot.writeI16((Short) value); + } + else if (value instanceof Integer) { + oprot.writeFieldBegin(new TField("dummy", TType.I32, thriftId)); + oprot.writeI32((Integer) value); + } + else if (value instanceof Long) { + oprot.writeFieldBegin(new TField("dummy", TType.I64, thriftId)); + oprot.writeI64((Long) value); + } + else if (value instanceof Double) { + oprot.writeFieldBegin(new TField("dummy", TType.DOUBLE, thriftId)); + oprot.writeDouble((Double) value); + } + else if (value instanceof String) { + oprot.writeFieldBegin(new TField("dummy", TType.STRING, thriftId)); + oprot.writeString((String) value); + } + else if (value instanceof ThriftGenericRow) { + oprot.writeFieldBegin(new TField("dummy", TType.STRUCT, thriftId)); + ((ThriftGenericRow) value).write(oprot); + } + else if (value instanceof List) { + oprot.writeFieldBegin(new TField("dummy", TType.LIST, thriftId)); + writeListField((List) value, oprot); + } + else if (value instanceof Set) { + oprot.writeFieldBegin(new TField("dummy", TType.SET, thriftId)); + writeSetField((Set) value, oprot); + } + else if (value instanceof Map) { + oprot.writeFieldBegin(new TField("dummy", TType.MAP, thriftId)); + writeMapField((Map) value, oprot); + } + oprot.writeFieldEnd(); + } + catch (TException e) { + throw new IllegalStateException(e); + } + } + + private void writeListField(List listValue, TProtocol oprot) throws TException + { + throw new UnsupportedOperationException("writeListField is not supported."); + } + + private void writeSetField(Set setValue, TProtocol oprot) throws TException + { + throw new UnsupportedOperationException("writeSetField is not supported."); + } + + private void writeMapField(Map setValue, TProtocol oprot) throws TException + { + throw new UnsupportedOperationException("writeMapField is not supported."); } public int compareTo(ThriftGenericRow other) 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 fa0fdadefdc0..053e8889bc1c 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 @@ -25,6 +25,9 @@ import com.facebook.presto.spi.RecordPageSource; import com.facebook.presto.spi.predicate.TupleDomain; 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; @@ -67,6 +70,7 @@ import static com.facebook.presto.hive.HiveStorageFormat.RCTEXT; import static com.facebook.presto.hive.HiveStorageFormat.SEQUENCEFILE; import static com.facebook.presto.hive.HiveStorageFormat.TEXTFILE; +import static com.facebook.presto.hive.HiveStorageFormat.THRIFTBINARY; import static com.facebook.presto.hive.HiveTestUtils.HDFS_ENVIRONMENT; import static com.facebook.presto.hive.HiveTestUtils.SESSION; import static com.facebook.presto.hive.HiveTestUtils.TYPE_MANAGER; @@ -83,6 +87,7 @@ import static java.util.Objects.requireNonNull; import static java.util.stream.Collectors.toList; import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.FILE_INPUT_FORMAT; +import static org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS; import static org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_LIB; import static org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.getStandardListObjectInspector; import static org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.getStandardStructObjectInspector; @@ -491,6 +496,54 @@ public void testParquetThrift(int rowCount) testCursorProvider(cursorProvider, split, PARQUET, testColumns, 1); } + @Test(dataProvider = "rowCount") + public void testLZOThrift(int rowCount) + throws Exception + { + RowType nameType = new RowType(ImmutableList.of(createUnboundedVarcharType(), createUnboundedVarcharType()), Optional.empty()); + RowType phoneType = new RowType(ImmutableList.of(createUnboundedVarcharType(), createUnboundedVarcharType()), Optional.empty()); + RowType personType = new RowType(ImmutableList.of(nameType, INTEGER, createUnboundedVarcharType(), new ArrayType(phoneType)), Optional.empty()); + + List testColumns = ImmutableList.of( + new TestColumn( + "persons", + getStandardListObjectInspector( + getStandardStructObjectInspector( + ImmutableList.of("name", "id", "email", "phones"), + ImmutableList.of( + getStandardStructObjectInspector( + ImmutableList.of("first_name", "last_name"), + ImmutableList.of(javaStringObjectInspector, javaStringObjectInspector) + ), + javaIntObjectInspector, + javaStringObjectInspector, + getStandardListObjectInspector( + getStandardStructObjectInspector( + ImmutableList.of("number", "type"), + ImmutableList.of(javaStringObjectInspector, javaStringObjectInspector) + ) + ) + ) + ) + ), + null, + arrayBlockOf(personType, + rowBlockOf(ImmutableList.of(nameType, INTEGER, createUnboundedVarcharType(), new ArrayType(phoneType)), + rowBlockOf(ImmutableList.of(createUnboundedVarcharType(), createUnboundedVarcharType()), "Bob", "Roberts"), + 0, + "bob.roberts@example.com", + arrayBlockOf(phoneType, rowBlockOf(ImmutableList.of(createUnboundedVarcharType(), createUnboundedVarcharType()), "1234567890", null)) + ) + ) + ) + ); + + File file = new File(this.getClass().getClassLoader().getResource("addressbook.thrift.lzo").getPath()); + FileSplit split = new FileSplit(new Path(file.getAbsolutePath()), 0, file.length(), new String[0]); + HiveRecordCursorProvider cursorProvider = new ThriftHiveRecordCursorProvider(HDFS_ENVIRONMENT, new HiveThriftFieldIdResolverFactory()); + testCursorProvider(cursorProvider, split, THRIFTBINARY, testColumns, 1); + } + @Test(dataProvider = "rowCount") public void testDwrf(int rowCount) throws Exception @@ -631,6 +684,9 @@ private void testCursorProvider(HiveRecordCursorProvider cursorProvider, splitProperties.setProperty(SERIALIZATION_LIB, storageFormat.getSerDe()); splitProperties.setProperty("columns", Joiner.on(',').join(transform(filter(testColumns, not(TestColumn::isPartitionKey)), TestColumn::getName))); splitProperties.setProperty("columns.types", Joiner.on(',').join(transform(filter(testColumns, not(TestColumn::isPartitionKey)), TestColumn::getType))); + if (storageFormat.equals(THRIFTBINARY)) { + splitProperties.setProperty(SERIALIZATION_CLASS, ThriftGenericRow.class.getName()); + } List partitionKeys = testColumns.stream() .filter(TestColumn::isPartitionKey) From c526b86cafec99d7f21fde6032854d6390e8fa77 Mon Sep 17 00:00:00 2001 From: thomass Date: Wed, 19 Apr 2017 23:42:51 -0700 Subject: [PATCH 168/331] resolve dependency conflicts and revert version back to twitter version --- pom.xml | 4 +-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 9 ++++- presto-hive-cdh5/pom.xml | 9 ++++- presto-hive-hadoop1/pom.xml | 9 ++++- presto-hive-hadoop2/pom.xml | 9 ++++- presto-hive/pom.xml | 48 +++++++++++++------------- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 +-- 48 files changed, 101 insertions(+), 73 deletions(-) diff --git a/pom.xml b/pom.xml index 5955f2967cbc..468980803c18 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.173 + 0.170-tw-0.32 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index cd3c9fe6593d..a37ea5c2f96b 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 148f63c1c244..d7f7024682e3 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 43c22888b4ec..31680c745445 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index e149d1896e6a..ecf712772755 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index ad92ffb73fcf..362b1f8f3241 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 30db245cb4bb..e922fcbb916a 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.173 + 0.170-tw-0.32 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index dddfb20dbecf..28e06f029e58 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index bc75462e53ba..c2b8f0c378eb 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index e3c9f43dfef1..9b0d4f9f79d8 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 5cc5401e2662..289cad7109fa 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index f7d5a5cebbcd..61c6623feebd 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index fe07db0f630e..2f5114c3a4a7 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 03ca2c3cb480..1981cf750aa0 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index a184c81691e5..9a6a0724f65f 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index d3cc8ad84974..c514ed1075b6 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-hive-cdh4 @@ -60,6 +60,13 @@ + + org.apache.zookeeper + zookeeper + 3.5.1-alpha + test + + org.testng testng diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index f7072fd6c973..3e914eb73a46 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-hive-cdh5 @@ -60,6 +60,13 @@ + + org.apache.zookeeper + zookeeper + 3.5.1-alpha + test + + org.testng testng diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 11598d13a3aa..71bc4b5b1925 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-hive-hadoop1 @@ -60,6 +60,13 @@ + + org.apache.zookeeper + zookeeper + 3.5.1-alpha + test + + org.testng testng diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index f3dc07fd0d26..479cad532bc1 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-hive-hadoop2 @@ -60,6 +60,13 @@ + + org.apache.zookeeper + zookeeper + 3.5.1-alpha + test + + org.testng testng diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 3157e6289b2b..77e868ca5b9c 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-hive @@ -51,39 +51,19 @@ org.apache.curator curator-recipes - 2.8.0 + 3.3.0 org.apache.curator curator-framework - 2.8.0 + 3.3.0 org.apache.curator curator-client - 2.8.0 - - - - org.apache.curator - curator-test - 2.8.0 - test - - - - org.apache.zookeeper - zookeeper - 3.4.6 - test - - - - com.101tec - zkclient - test + 3.3.0 @@ -243,6 +223,26 @@ + + org.apache.curator + curator-test + 3.3.0 + test + + + + org.apache.zookeeper + zookeeper + 3.5.1-alpha + test + + + + com.101tec + zkclient + test + + com.facebook.presto presto-main diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index ebac6c7cac8d..443e93bfa2bb 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 93b93454a96c..7dc9f4e215ad 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 441df3abbd17..22b22b95232f 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 5505d7ad0506..ceecc3da62a3 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 82201b31dd1b..bb42e26c4f57 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-main diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 120ef8203efa..2a634850be7f 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index fd182dfb964a..ac06d2ea198f 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index e9325bb1a53c..062ee8c05e8f 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 154308c7a3b1..2a9879c22f1c 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 0e6aaf43dfe9..88d5a99cfe79 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 481b34a63f5f..8c6fae50a5a7 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index ff76ac8cb3fb..ddf520bebb66 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index d2d08db8b05a..1dba94f35a14 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 46192eeca751..2c837b379e1d 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.173 + 0.170-tw-0.32 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index d4ad9695363a..4f83793ca631 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index a4abf7e454b4..00f49ad4d0db 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 32484a240a34..6db7c2373423 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 6887cee87173..974ffffabd0a 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index f16790697439..a6a641bdab23 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 765195387829..a2b420c7465c 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index fea50eb030c6..5c76a7a23ba2 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 586912612e54..b2853ac43c50 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index a4397c2fc0c0..ef8c6726398d 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index a3d1b570fc25..0d058bb07f73 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index cbcccf2db6f0..55815bfae2f2 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.173 + 0.170-tw-0.32 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 45137f24398f..dfd00558a2a5 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index d23c73c4b4e7..d09e79b34b23 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 7f8b2c7dc6b3..f404c698d7dd 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173 + 0.170-tw-0.32 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.173 + 0.170-tw-0.32 provided From e6025e5de85e1e16102b021ee726f4b5d6bb90a9 Mon Sep 17 00:00:00 2001 From: thomass Date: Fri, 21 Apr 2017 12:16:17 -0700 Subject: [PATCH 169/331] exclude zookeeper from presto-hive instead of adding it with the same version --- presto-hive-cdh4/pom.xml | 19 ++++++++++++------- presto-hive-cdh5/pom.xml | 19 ++++++++++++------- presto-hive-hadoop1/pom.xml | 19 ++++++++++++------- presto-hive-hadoop2/pom.xml | 19 ++++++++++++------- 4 files changed, 48 insertions(+), 28 deletions(-) diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index c514ed1075b6..5b3a10814348 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -20,6 +20,12 @@ com.facebook.presto presto-hive + + + org.apache.zookeeper + zookeeper + + @@ -60,13 +66,6 @@ - - org.apache.zookeeper - zookeeper - 3.5.1-alpha - test - - org.testng testng @@ -82,6 +81,12 @@ com.facebook.presto presto-hive + + + org.apache.zookeeper + zookeeper + + test-jar test diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 3e914eb73a46..a859111fa7df 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -20,6 +20,12 @@ com.facebook.presto presto-hive + + + org.apache.zookeeper + zookeeper + + @@ -60,13 +66,6 @@ - - org.apache.zookeeper - zookeeper - 3.5.1-alpha - test - - org.testng testng @@ -82,6 +81,12 @@ com.facebook.presto presto-hive + + + org.apache.zookeeper + zookeeper + + test-jar test diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 71bc4b5b1925..88289803fef3 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -20,6 +20,12 @@ com.facebook.presto presto-hive + + + org.apache.zookeeper + zookeeper + + @@ -60,13 +66,6 @@ - - org.apache.zookeeper - zookeeper - 3.5.1-alpha - test - - org.testng testng @@ -82,6 +81,12 @@ com.facebook.presto presto-hive + + + org.apache.zookeeper + zookeeper + + test-jar test diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 479cad532bc1..a095a96b22ad 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -20,6 +20,12 @@ com.facebook.presto presto-hive + + + org.apache.zookeeper + zookeeper + + @@ -60,13 +66,6 @@ - - org.apache.zookeeper - zookeeper - 3.5.1-alpha - test - - org.testng testng @@ -82,6 +81,12 @@ com.facebook.presto presto-hive + + + org.apache.zookeeper + zookeeper + + test-jar test From b914b617169e1449a519b5a0b9e137153f96d38a Mon Sep 17 00:00:00 2001 From: thomass Date: Tue, 25 Apr 2017 15:16:23 -0700 Subject: [PATCH 170/331] use 3.5.1-alpha for zookeeper overall and 3.4.9 for kafka; use guava 21.0 in presto-tests to avoid clientcxn oom errors --- pom.xml | 2 +- presto-hive-cdh4/pom.xml | 12 ------------ presto-hive-cdh5/pom.xml | 12 ------------ presto-hive-hadoop1/pom.xml | 12 ------------ presto-hive-hadoop2/pom.xml | 12 ------------ presto-hive/pom.xml | 1 - presto-kafka/pom.xml | 1 + presto-tests/pom.xml | 1 + 8 files changed, 3 insertions(+), 50 deletions(-) diff --git a/pom.xml b/pom.xml index 468980803c18..b7b0eee87f93 100644 --- a/pom.xml +++ b/pom.xml @@ -710,7 +710,7 @@ org.apache.zookeeper zookeeper - 3.4.9 + 3.5.1-alpha jline diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 5b3a10814348..e31ee043a73b 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -20,12 +20,6 @@ com.facebook.presto presto-hive - - - org.apache.zookeeper - zookeeper - - @@ -81,12 +75,6 @@ com.facebook.presto presto-hive - - - org.apache.zookeeper - zookeeper - - test-jar test diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index a859111fa7df..2e01b0fd9bcd 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -20,12 +20,6 @@ com.facebook.presto presto-hive - - - org.apache.zookeeper - zookeeper - - @@ -81,12 +75,6 @@ com.facebook.presto presto-hive - - - org.apache.zookeeper - zookeeper - - test-jar test diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 88289803fef3..b18f5fd4d701 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -20,12 +20,6 @@ com.facebook.presto presto-hive - - - org.apache.zookeeper - zookeeper - - @@ -81,12 +75,6 @@ com.facebook.presto presto-hive - - - org.apache.zookeeper - zookeeper - - test-jar test diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index a095a96b22ad..4cdfbe145f3d 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -20,12 +20,6 @@ com.facebook.presto presto-hive - - - org.apache.zookeeper - zookeeper - - @@ -81,12 +75,6 @@ com.facebook.presto presto-hive - - - org.apache.zookeeper - zookeeper - - test-jar test diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 77e868ca5b9c..abeedcec79d7 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -233,7 +233,6 @@ org.apache.zookeeper zookeeper - 3.5.1-alpha test diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 22b22b95232f..e68a4a079574 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -165,6 +165,7 @@ org.apache.zookeeper zookeeper + 3.4.9 test diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 55815bfae2f2..7e84577b64a6 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -135,6 +135,7 @@ com.google.guava guava + 21.0 From 49e673d2ac11dde75ea005e97b2a03ff4fe12494 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 25 Apr 2017 18:16:54 -0700 Subject: [PATCH 171/331] add support to split lzo file in createHiveSplitIterator directly --- .../hive/BackgroundHiveSplitLoader.java | 129 +++++------------- .../com/facebook/presto/hive/HiveUtil.java | 25 +++- 2 files changed, 52 insertions(+), 102 deletions(-) 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 ff5916b0631f..e66738e7fc6e 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 @@ -24,7 +24,6 @@ import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.StandardErrorCode; import com.facebook.presto.spi.predicate.TupleDomain; -import com.facebook.presto.twitter.hive.thrift.ThriftGeneralInputFormat; import com.facebook.presto.twitter.hive.util.UgiUtils; import com.google.common.base.Throwables; import com.google.common.collect.AbstractIterator; @@ -58,6 +57,7 @@ import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.Deque; import java.util.Iterator; import java.util.List; @@ -81,7 +81,9 @@ import static com.facebook.presto.hive.HiveSessionProperties.getMaxSplitSize; import static com.facebook.presto.hive.HiveUtil.checkCondition; import static com.facebook.presto.hive.HiveUtil.getInputFormat; +import static com.facebook.presto.hive.HiveUtil.getLzoIndexPath; import static com.facebook.presto.hive.HiveUtil.isLzoCompressedFile; +import static com.facebook.presto.hive.HiveUtil.isLzoIndexFile; import static com.facebook.presto.hive.HiveUtil.isSplittable; import static com.facebook.presto.hive.metastore.MetastoreUtil.getHiveSchema; import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; @@ -89,7 +91,6 @@ import static java.lang.Math.toIntExact; import static java.lang.String.format; import static org.apache.hadoop.hive.common.FileUtils.HIDDEN_FILES_PATH_FILTER; -import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_NAME; public class BackgroundHiveSplitLoader implements HiveSplitLoader @@ -366,11 +367,6 @@ private void loadPartition(HivePartitionMetadata partition) // If only one bucket could match: load that one file HiveFileIterator iterator = new HiveFileIterator(path, fs, directoryLister, namenodeStats, partitionName, inputFormat, schema, partitionKeys, effectivePredicate, partition.getColumnCoercions()); - if (inputFormat instanceof ThriftGeneralInputFormat) { - addLzoThriftSplitsToQueue(iterator, partitionName, schema, partitionKeys, session, effectivePredicate, partition.getColumnCoercions()); - return; - } - if (!buckets.isEmpty()) { int bucketCount = buckets.get(0).getBucketCount(); List list = listAndSortBucketFiles(iterator, bucketCount); @@ -433,88 +429,6 @@ private void loadPartition(HivePartitionMetadata partition) fileIterators.addLast(iterator); } - private void addLzoThriftSplitsToQueue( - HiveFileIterator hiveFileIterator, - String partitionName, - Properties schema, - List partitionKeys, - ConnectorSession session, - TupleDomain effectivePredicate, - Map columnCoercions) - throws IOException - { - if (bucketHandle.isPresent()) { - throw new PrestoException(StandardErrorCode.NOT_SUPPORTED, format("Bucketed table %s in ThriftGeneralInputFormat is not yet supported", schema.getProperty(META_TABLE_NAME))); - } - List> iteratorList = new ArrayList<>(); - - while (hiveFileIterator.hasNext()) { - LocatedFileStatus file = hiveFileIterator.next(); - if (!isLzoCompressedFile(file.getPath())) { - continue; - } - InputSplit[] targetSplits = createLzoSplits(file); - log.debug("For file at %s, get number of splits %s", file.getPath(), targetSplits.length); - FileSystem targetFilesystem = hdfsEnvironment.getFileSystem(session.getUser(), file.getPath()); - FileStatus targetFile = targetFilesystem.getFileStatus(file.getPath()); - for (InputSplit inputSplit : targetSplits) { - FileSplit split = (FileSplit) inputSplit; - iteratorList.add(createHiveSplitIterator( - partitionName, - targetFile.getPath().toString(), - targetFilesystem.getFileBlockLocations(file, split.getStart(), split.getLength()), - split.getStart(), - split.getLength(), - schema, - partitionKeys, - false, - session, - OptionalInt.empty(), - effectivePredicate, - columnCoercions)); - if (stopped) { - return; - } - } - } - - addToHiveSplitSourceRoundRobin(iteratorList); - } - - private InputSplit[] createLzoSplits(LocatedFileStatus file) - throws IOException - { - Configuration targetConfiguration = hdfsEnvironment.getConfiguration(file.getPath()); - JobConf job = new JobConf(targetConfiguration); - FileSystem indexFilesystem = hdfsEnvironment.getFileSystem(session.getUser(), getLzoIndexPath(file.getPath())); - LzoIndex index = LzoIndex.readIndex(indexFilesystem, file.getPath()); - if (index.isEmpty()) { - InputSplit[] splits = new InputSplit[1]; - splits[0] = new FileSplit(file.getPath(), 0, file.getLen(), job); - return splits; - } - - List splits = new ArrayList<>(); - long chunkOffset = 0; - while (chunkOffset < file.getLen()) { - long targetChunkSize; - if (remainingInitialSplits.decrementAndGet() >= 0) { - targetChunkSize = maxInitialSplitSize.toBytes(); - } - else { - long maxBytes = maxSplitSize.toBytes(); - int chunks = toIntExact((long) Math.ceil((file.getLen() - chunkOffset) * 1.0 / maxBytes)); - targetChunkSize = (long) Math.ceil((file.getLen() - chunkOffset) * 1.0 / chunks); - } - long chunkEnd = index.alignSliceEndToIndex(chunkOffset + targetChunkSize, file.getLen()); - - splits.add(new FileSplit(file.getPath(), chunkOffset, chunkEnd - chunkOffset, job)); - chunkOffset = chunkEnd; - } - - return splits.toArray(new InputSplit[0]); - } - private boolean addSplitsToSource( InputSplit[] targetSplits, String partitionName, @@ -629,6 +543,11 @@ private Iterator createHiveSplitIterator( Map columnCoercions) throws IOException { + Path filePath = new Path(path); + // filter the index files + if (isLzoIndexFile(filePath)) { + return Collections.emptyIterator(); + } boolean forceLocalScheduling = HiveSessionProperties.isForceLocalScheduling(session); if (splittable) { @@ -636,6 +555,9 @@ private Iterator createHiveSplitIterator( return new AbstractIterator() { private long chunkOffset = 0; + private LzoIndex index = isLzoCompressedFile(filePath) ? + LzoIndex.readIndex(hdfsEnvironment.getFileSystem(session.getUser(), getLzoIndexPath(filePath)), filePath) : + null; @Override protected HiveSplit computeNext() @@ -666,6 +588,18 @@ protected HiveSplit computeNext() // adjust the actual chunk size to account for the overrun when chunks are slightly bigger than necessary (see above) long chunkLength = Math.min(targetChunkSize, blockLocation.getLength() - chunkOffset); + // align the end point to the indexed point for lzo compressed file + if (isLzoCompressedFile(filePath)) { + long offset = blockLocation.getOffset() + chunkOffset; + if (index.isEmpty()) { + chunkLength = length - offset; + } + else { + chunkLength = index.alignSliceEndToIndex(offset + chunkLength, length) - offset; + } + log.debug("lzo split: %s (%s:%s)", path, offset, offset + chunkLength); + } + HiveSplit result = new HiveSplit( connectorId, table.getDatabaseName(), @@ -684,10 +618,16 @@ protected HiveSplit computeNext() chunkOffset += chunkLength; - if (chunkOffset >= blockLocation.getLength()) { - checkState(chunkOffset == blockLocation.getLength(), "Error splitting blocks"); + while (chunkOffset >= blockLocation.getLength()) { + // allow overrun for lzo compressed file for intermediate blocks + if (!isLzoCompressedFile(filePath) || blockLocation.getOffset() + blockLocation.getLength() >= length) { + checkState(chunkOffset == blockLocation.getLength(), "Error splitting blocks"); + } blockLocationIterator.next(); - chunkOffset = 0; + chunkOffset -= blockLocation.getLength(); + if (blockLocationIterator.hasNext()) { + blockLocation = blockLocationIterator.peek(); + } } return result; @@ -771,9 +711,4 @@ private static String getPartitionLocation(Table table, Optional part } return partition.get().getStorage().getLocation(); } - - private static Path getLzoIndexPath(Path lzoPath) - { - return lzoPath.suffix(LzoIndex.LZO_INDEX_SUFFIX); - } } 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 e4d401aac30f..c396fdeca427 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 @@ -107,6 +107,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.Iterables.filter; import static com.google.common.collect.Lists.transform; +import static com.hadoop.compression.lzo.LzoIndex.LZO_INDEX_SUFFIX; import static java.lang.Byte.parseByte; import static java.lang.Double.parseDouble; import static java.lang.Float.floatToRawIntBits; @@ -151,6 +152,14 @@ public boolean accept(Path path) } }; + private static final PathFilter LZOP_INDEX_DEFAULT_SUFFIX_FILTER = new PathFilter() { + @Override + public boolean accept(Path path) + { + return path.toString().endsWith(".lzo.index"); + } + }; + static { DateTimeParser[] timestampWithoutTimeZoneParser = { DateTimeFormat.forPattern("yyyy-M-d").getParser(), @@ -278,10 +287,6 @@ static boolean isSplittable(InputFormat inputFormat, FileSystem fileSystem return true; } - if (inputFormat.getClass().getSimpleName().equals("ThriftGeneralInputFormat")) { - return false; - } - // use reflection to get isSplittable method on FileInputFormat Method method = null; for (Class clazz = inputFormat.getClass(); clazz != null; clazz = clazz.getSuperclass()) { @@ -305,11 +310,21 @@ static boolean isSplittable(InputFormat inputFormat, FileSystem fileSystem } } - static boolean isLzoCompressedFile(Path filePath) + public static boolean isLzoCompressedFile(Path filePath) { return LZOP_DEFAULT_SUFFIX_FILTER.accept(filePath); } + public static boolean isLzoIndexFile(Path filePath) + { + return LZOP_INDEX_DEFAULT_SUFFIX_FILTER.accept(filePath); + } + + public static Path getLzoIndexPath(Path lzoPath) + { + return lzoPath.suffix(LZO_INDEX_SUFFIX); + } + public static StructObjectInspector getTableObjectInspector(Properties schema) { return getTableObjectInspector(getDeserializer(schema)); From bde735a641373817c924ff29222f8bf5aff083b0 Mon Sep 17 00:00:00 2001 From: thomass Date: Tue, 25 Apr 2017 18:44:52 -0700 Subject: [PATCH 172/331] add TODO comments to adapt better versions of packages to resolve dep conflicts --- pom.xml | 2 ++ presto-kafka/pom.xml | 1 + presto-tests/pom.xml | 1 + 3 files changed, 4 insertions(+) diff --git a/pom.xml b/pom.xml index b7b0eee87f93..9cae7c5a9fea 100644 --- a/pom.xml +++ b/pom.xml @@ -710,6 +710,8 @@ org.apache.zookeeper zookeeper + + 3.5.1-alpha diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index e68a4a079574..851fb693ca79 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -165,6 +165,7 @@ org.apache.zookeeper zookeeper + 3.4.9 test diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 7e84577b64a6..371dba88b1f8 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -135,6 +135,7 @@ com.google.guava guava + 21.0 From fc43cc4960dcee275d4974c69a2aa2672a4bb039 Mon Sep 17 00:00:00 2001 From: thomass Date: Tue, 25 Apr 2017 18:51:49 -0700 Subject: [PATCH 173/331] Make internal release tag 0.173-tw-0.33 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 48 files changed, 50 insertions(+), 50 deletions(-) diff --git a/pom.xml b/pom.xml index 9cae7c5a9fea..80cbb43f0a05 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.170-tw-0.32 + 0.173-tw-0.33 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index a37ea5c2f96b..57f73633e6c6 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index d7f7024682e3..7a73340e18a3 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 31680c745445..8dabe89e1c0e 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index ecf712772755..579d20650b04 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 362b1f8f3241..934671a3c775 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index e922fcbb916a..304e776e7995 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.170-tw-0.32 + 0.173-tw-0.33 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 28e06f029e58..72e9190b9ad3 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index c2b8f0c378eb..842bd187854d 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 9b0d4f9f79d8..5a173e2f14a4 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 289cad7109fa..f9be10f853f8 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 61c6623feebd..97f7c3fd38ac 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 2f5114c3a4a7..6ba1a5a5c1f7 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 1981cf750aa0..06b98fe55d83 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 9a6a0724f65f..9e6511b677f9 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index e31ee043a73b..aca7023060a6 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index 2e01b0fd9bcd..d4f72665764d 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index b18f5fd4d701..35c0556bb36a 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 4cdfbe145f3d..76052ec651ec 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index abeedcec79d7..7d6dc7d188f7 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 443e93bfa2bb..62b53288395b 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 7dc9f4e215ad..cd8369eebdd2 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 851fb693ca79..75fcfd30e054 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index ceecc3da62a3..44fc6ddd5d47 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index bb42e26c4f57..57f5031f4093 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-main diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 2a634850be7f..6d45014b6c74 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index ac06d2ea198f..bf4f00ea8ba4 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 062ee8c05e8f..99abe5975afe 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 2a9879c22f1c..10c0b359fa0d 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 88d5a99cfe79..56a1b6e9566f 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 8c6fae50a5a7..eaa09a66ccf7 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index ddf520bebb66..025cfd8abaf0 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 1dba94f35a14..293868d3f2f5 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 2c837b379e1d..ad10c409d538 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.170-tw-0.32 + 0.173-tw-0.33 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 4f83793ca631..2ce7343b05d2 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 00f49ad4d0db..0595afa11b3d 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 6db7c2373423..4d6ca18c3f68 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 974ffffabd0a..0699e33d8b67 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index a6a641bdab23..470a294e8f25 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index a2b420c7465c..a5e8abb07a1a 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 5c76a7a23ba2..ed2ee4a8b3ea 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index b2853ac43c50..023f0c10afd9 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index ef8c6726398d..3ce5976946fa 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 0d058bb07f73..14c48cba3d4a 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 371dba88b1f8..d7cbc61f9569 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.170-tw-0.32 + 0.173-tw-0.33 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index dfd00558a2a5..e396c3e8b368 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index d09e79b34b23..36bd77a7bbdd 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index f404c698d7dd..1f1040072fef 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.170-tw-0.32 + 0.173-tw-0.33 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.170-tw-0.32 + 0.173-tw-0.33 provided From d0d2ae3f70269d3605bc29a12b24dd8b87a5e1d6 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 26 Apr 2017 17:17:29 -0700 Subject: [PATCH 174/331] Added unit test for pre-generated lzo thrift data --- .../facebook/presto/hive/TestHiveFileFormats.java | 6 +++++- .../src/test/resources/addressbook.thrift.lzo | Bin 0 -> 204 bytes 2 files changed, 5 insertions(+), 1 deletion(-) create mode 100644 presto-hive/src/test/resources/addressbook.thrift.lzo 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 053e8889bc1c..30cd3540032d 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 @@ -687,6 +687,10 @@ private void testCursorProvider(HiveRecordCursorProvider cursorProvider, if (storageFormat.equals(THRIFTBINARY)) { splitProperties.setProperty(SERIALIZATION_CLASS, ThriftGenericRow.class.getName()); } + Configuration configuration = new Configuration(); + if (storageFormat.equals(THRIFTBINARY)) { + configuration.set("io.compression.codecs", "com.hadoop.compression.lzo.LzoCodec,com.hadoop.compression.lzo.LzopCodec"); + } List partitionKeys = testColumns.stream() .filter(TestColumn::isPartitionKey) @@ -697,7 +701,7 @@ private void testCursorProvider(HiveRecordCursorProvider cursorProvider, ImmutableSet.of(cursorProvider), ImmutableSet.of(), "test", - new Configuration(), + configuration, SESSION, split.getPath(), OptionalInt.empty(), diff --git a/presto-hive/src/test/resources/addressbook.thrift.lzo b/presto-hive/src/test/resources/addressbook.thrift.lzo new file mode 100644 index 0000000000000000000000000000000000000000..bc982749d3ff979cd0fe53d30ca3ad1dc92727d4 GIT binary patch literal 204 zcmeD5iSlRQ<&xqO5Ku7SbYNy@0E5OQk&LpoV5Z_Hrnx}IN+6!)t$E`rTf|u(%}sm4 zXDSx%{XZ2b%E2h4m7JfeS5lr?Qj%JvS5TB%T#~O>l2MeIR#Kdtn3JgIn358}#8K>& zpPwz2%Fn>a6Tue5z{t(P;L6C%%bLW>#N@`v9>m6!TExW5z%i4Nft8gxj6pn!ja9E` s2{W5RYDHphK~AckJ0Cv-O9ST?4h}9uBV!X&Gjj_|gAhhRpeuj?0Lut2L;wH) literal 0 HcmV?d00001 From 9b06ace8cac4583b6ebe7b7147d096a58e837bb1 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 26 Apr 2017 17:20:07 -0700 Subject: [PATCH 175/331] Remove log in BackgroundHiveSplitLoader --- .../com/facebook/presto/hive/BackgroundHiveSplitLoader.java | 3 --- 1 file changed, 3 deletions(-) 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 e66738e7fc6e..f7615877b3e4 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 @@ -32,7 +32,6 @@ import com.google.common.collect.PeekingIterator; import com.google.common.io.CharStreams; import com.hadoop.compression.lzo.LzoIndex; -import io.airlift.log.Logger; import io.airlift.units.DataSize; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BlockLocation; @@ -96,7 +95,6 @@ public class BackgroundHiveSplitLoader implements HiveSplitLoader { private static final String CORRUPT_BUCKETING = "Hive table is corrupt. It is declared as being bucketed, but the files do not match the bucketing declaration."; - private static final Logger log = Logger.get(BackgroundHiveSplitLoader.class); public static final CompletableFuture COMPLETED_FUTURE = CompletableFuture.completedFuture(null); @@ -597,7 +595,6 @@ protected HiveSplit computeNext() else { chunkLength = index.alignSliceEndToIndex(offset + chunkLength, length) - offset; } - log.debug("lzo split: %s (%s:%s)", path, offset, offset + chunkLength); } HiveSplit result = new HiveSplit( From 0a648af7561b0a3c1e2952e2dc9dc5af8a131057 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 26 Apr 2017 17:40:41 -0700 Subject: [PATCH 176/331] change the writeField parameter in ThriftGenericRow --- .../twitter/hive/thrift/ThriftGenericRow.java | 28 +++++++++---------- 1 file changed, 13 insertions(+), 15 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java index 559fe7825756..733391ed6736 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java @@ -222,58 +222,56 @@ public void write(TProtocol oprot) throws TException oprot.writeStructBegin(new TStruct("dummy")); values.entrySet().stream() .filter(v -> v.getValue() != null) - .forEach(v -> writeField(v, oprot)); + .forEach(v -> writeField(v.getKey(), v.getValue(), oprot)); oprot.writeFieldStop(); oprot.writeStructEnd(); } - private void writeField(Map.Entry field, TProtocol oprot) + private void writeField(short thriftId, Object value, TProtocol oprot) { - Object value = field.getValue(); - short thriftId = field.getKey(); try { if (value instanceof Boolean) { - oprot.writeFieldBegin(new TField("dummy", TType.BOOL, thriftId)); + oprot.writeFieldBegin(new TField("", TType.BOOL, thriftId)); oprot.writeBool((Boolean) value); } else if (value instanceof Byte) { - oprot.writeFieldBegin(new TField("dummy", TType.BYTE, thriftId)); + oprot.writeFieldBegin(new TField("", TType.BYTE, thriftId)); oprot.writeByte((Byte) value); } else if (value instanceof Short) { - oprot.writeFieldBegin(new TField("dummy", TType.I16, thriftId)); + oprot.writeFieldBegin(new TField("", TType.I16, thriftId)); oprot.writeI16((Short) value); } else if (value instanceof Integer) { - oprot.writeFieldBegin(new TField("dummy", TType.I32, thriftId)); + oprot.writeFieldBegin(new TField("", TType.I32, thriftId)); oprot.writeI32((Integer) value); } else if (value instanceof Long) { - oprot.writeFieldBegin(new TField("dummy", TType.I64, thriftId)); + oprot.writeFieldBegin(new TField("", TType.I64, thriftId)); oprot.writeI64((Long) value); } else if (value instanceof Double) { - oprot.writeFieldBegin(new TField("dummy", TType.DOUBLE, thriftId)); + oprot.writeFieldBegin(new TField("", TType.DOUBLE, thriftId)); oprot.writeDouble((Double) value); } else if (value instanceof String) { - oprot.writeFieldBegin(new TField("dummy", TType.STRING, thriftId)); + oprot.writeFieldBegin(new TField("", TType.STRING, thriftId)); oprot.writeString((String) value); } else if (value instanceof ThriftGenericRow) { - oprot.writeFieldBegin(new TField("dummy", TType.STRUCT, thriftId)); + oprot.writeFieldBegin(new TField("", TType.STRUCT, thriftId)); ((ThriftGenericRow) value).write(oprot); } else if (value instanceof List) { - oprot.writeFieldBegin(new TField("dummy", TType.LIST, thriftId)); + oprot.writeFieldBegin(new TField("", TType.LIST, thriftId)); writeListField((List) value, oprot); } else if (value instanceof Set) { - oprot.writeFieldBegin(new TField("dummy", TType.SET, thriftId)); + oprot.writeFieldBegin(new TField("", TType.SET, thriftId)); writeSetField((Set) value, oprot); } else if (value instanceof Map) { - oprot.writeFieldBegin(new TField("dummy", TType.MAP, thriftId)); + oprot.writeFieldBegin(new TField("", TType.MAP, thriftId)); writeMapField((Map) value, oprot); } oprot.writeFieldEnd(); From 85f376483e910f7dc7786f5eb3f53d682a918c66 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 27 Apr 2017 14:57:38 -0700 Subject: [PATCH 177/331] Adapt method name lzo->lzop as oss patch --- .../presto/hive/BackgroundHiveSplitLoader.java | 16 ++++++++-------- .../java/com/facebook/presto/hive/HiveUtil.java | 6 +++--- 2 files changed, 11 insertions(+), 11 deletions(-) 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 f7615877b3e4..8abfb900f608 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 @@ -80,9 +80,9 @@ import static com.facebook.presto.hive.HiveSessionProperties.getMaxSplitSize; import static com.facebook.presto.hive.HiveUtil.checkCondition; import static com.facebook.presto.hive.HiveUtil.getInputFormat; -import static com.facebook.presto.hive.HiveUtil.getLzoIndexPath; -import static com.facebook.presto.hive.HiveUtil.isLzoCompressedFile; -import static com.facebook.presto.hive.HiveUtil.isLzoIndexFile; +import static com.facebook.presto.hive.HiveUtil.getLzopIndexPath; +import static com.facebook.presto.hive.HiveUtil.isLzopCompressedFile; +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.spi.StandardErrorCode.NOT_SUPPORTED; @@ -543,7 +543,7 @@ private Iterator createHiveSplitIterator( { Path filePath = new Path(path); // filter the index files - if (isLzoIndexFile(filePath)) { + if (isLzopIndexFile(filePath)) { return Collections.emptyIterator(); } boolean forceLocalScheduling = HiveSessionProperties.isForceLocalScheduling(session); @@ -553,8 +553,8 @@ private Iterator createHiveSplitIterator( return new AbstractIterator() { private long chunkOffset = 0; - private LzoIndex index = isLzoCompressedFile(filePath) ? - LzoIndex.readIndex(hdfsEnvironment.getFileSystem(session.getUser(), getLzoIndexPath(filePath)), filePath) : + private LzoIndex index = isLzopCompressedFile(filePath) ? + LzoIndex.readIndex(hdfsEnvironment.getFileSystem(session.getUser(), getLzopIndexPath(filePath)), filePath) : null; @Override @@ -587,7 +587,7 @@ protected HiveSplit computeNext() long chunkLength = Math.min(targetChunkSize, blockLocation.getLength() - chunkOffset); // align the end point to the indexed point for lzo compressed file - if (isLzoCompressedFile(filePath)) { + if (isLzopCompressedFile(filePath)) { long offset = blockLocation.getOffset() + chunkOffset; if (index.isEmpty()) { chunkLength = length - offset; @@ -617,7 +617,7 @@ protected HiveSplit computeNext() while (chunkOffset >= blockLocation.getLength()) { // allow overrun for lzo compressed file for intermediate blocks - if (!isLzoCompressedFile(filePath) || blockLocation.getOffset() + blockLocation.getLength() >= length) { + if (!isLzopCompressedFile(filePath) || blockLocation.getOffset() + blockLocation.getLength() >= length) { checkState(chunkOffset == blockLocation.getLength(), "Error splitting blocks"); } blockLocationIterator.next(); 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 c396fdeca427..5991063413c9 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 @@ -310,17 +310,17 @@ static boolean isSplittable(InputFormat inputFormat, FileSystem fileSystem } } - public static boolean isLzoCompressedFile(Path filePath) + public static boolean isLzopCompressedFile(Path filePath) { return LZOP_DEFAULT_SUFFIX_FILTER.accept(filePath); } - public static boolean isLzoIndexFile(Path filePath) + public static boolean isLzopIndexFile(Path filePath) { return LZOP_INDEX_DEFAULT_SUFFIX_FILTER.accept(filePath); } - public static Path getLzoIndexPath(Path lzoPath) + public static Path getLzopIndexPath(Path lzoPath) { return lzoPath.suffix(LZO_INDEX_SUFFIX); } From 87cf0c0d9d687a3999ba9760bb96d47a33007e22 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 27 Apr 2017 19:01:13 -0700 Subject: [PATCH 178/331] Remove hard coded class name and Remove LZO codec in HiveCompressionCodec --- .../com/facebook/presto/hive/HiveCompressionCodec.java | 4 +--- .../hive/thrift/ThriftHiveRecordCursorProvider.java | 9 ++++----- 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveCompressionCodec.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveCompressionCodec.java index de314b5fdbfc..34cea5b9d39d 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveCompressionCodec.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveCompressionCodec.java @@ -16,7 +16,6 @@ import org.apache.hadoop.hive.ql.io.orc.CompressionKind; import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.GzipCodec; -import org.apache.hadoop.io.compress.LzoCodec; import org.apache.hadoop.io.compress.SnappyCodec; import parquet.hadoop.metadata.CompressionCodecName; @@ -28,8 +27,7 @@ public enum HiveCompressionCodec { NONE(null, CompressionKind.NONE, CompressionCodecName.UNCOMPRESSED), SNAPPY(SnappyCodec.class, CompressionKind.SNAPPY, CompressionCodecName.SNAPPY), - GZIP(GzipCodec.class, CompressionKind.ZLIB, CompressionCodecName.GZIP), - LZO(LzoCodec.class, CompressionKind.LZO, CompressionCodecName.LZO); + GZIP(GzipCodec.class, CompressionKind.ZLIB, CompressionCodecName.GZIP); private final Optional> codec; private final CompressionKind orcCompressionKind; 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 10439e2b9d53..c2169b4b3946 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 @@ -34,6 +34,7 @@ import java.util.Properties; import java.util.Set; +import static com.facebook.presto.hive.HiveStorageFormat.THRIFTBINARY; import static com.facebook.presto.hive.HiveUtil.createRecordReader; import static com.facebook.presto.hive.HiveUtil.getDeserializerClassName; import static com.facebook.presto.hive.HiveUtil.getSerializationClassName; @@ -42,12 +43,10 @@ public class ThriftHiveRecordCursorProvider implements HiveRecordCursorProvider { - private static final String LAZY_BINARY_SERDE = "org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe"; private static final String THRIFT_GENERIC_ROW = ThriftGenericRow.class.getName(); private static final Set THRIFT_SERDE_CLASS_NAMES = ImmutableSet.builder() - .add("com.facebook.presto.twitter.hive.thrift.ThriftGeneralSerDe") - .add("com.facebook.presto.twitter.hive.thrift.ThriftGeneralDeserializer") - .add(LAZY_BINARY_SERDE) + .add(ThriftGeneralDeserializer.class.getName()) + .add(THRIFTBINARY.getSerDe()) .build(); private final HdfsEnvironment hdfsEnvironment; private final ThriftFieldIdResolverFactory thriftFieldIdResolverFactory; @@ -79,7 +78,7 @@ public Optional createRecordCursor( // We only allow the table which specified its serialization class is compatible to // our thrift general row, if the SerDe is LazyBinarySerDe. - if (LAZY_BINARY_SERDE.equals(getDeserializerClassName(schema)) && !THRIFT_GENERIC_ROW.equals(getSerializationClassName(schema))) { + if (THRIFTBINARY.getSerDe().equals(getDeserializerClassName(schema)) && !THRIFT_GENERIC_ROW.equals(getSerializationClassName(schema))) { return Optional.empty(); } From 21a2b5cd3dc8ad29f8e203033b0777959663c10f Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 27 Apr 2017 22:35:50 -0700 Subject: [PATCH 179/331] Remove tests on THRIFTBINARY format in AbstractTestHiveClient, TestHiveIntegrationSmokeTest, TestHivePageSink --- .../java/com/facebook/presto/hive/AbstractTestHiveClient.java | 3 ++- .../com/facebook/presto/hive/TestHiveIntegrationSmokeTest.java | 3 +++ .../test/java/com/facebook/presto/hive/TestHivePageSink.java | 3 +++ 3 files changed, 8 insertions(+), 1 deletion(-) 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 2337e98b472b..5a2eefe7868e 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 @@ -138,6 +138,7 @@ import static com.facebook.presto.hive.HiveStorageFormat.RCTEXT; import static com.facebook.presto.hive.HiveStorageFormat.SEQUENCEFILE; import static com.facebook.presto.hive.HiveStorageFormat.TEXTFILE; +import static com.facebook.presto.hive.HiveStorageFormat.THRIFTBINARY; import static com.facebook.presto.hive.HiveTableProperties.BUCKETED_BY_PROPERTY; import static com.facebook.presto.hive.HiveTableProperties.BUCKET_COUNT_PROPERTY; import static com.facebook.presto.hive.HiveTableProperties.PARTITIONED_BY_PROPERTY; @@ -313,7 +314,7 @@ public abstract class AbstractTestHiveClient .row(6L, "bye", (byte) 46, (short) 346, 345, 456L, -754.2008f, 98.1, false, ImmutableList.of("ape", "bear"), ImmutableMap.of("three", 3L, "four", 4L), ImmutableList.of("false", 0L, false), "2015-07-04") .build(); - protected Set createTableFormats = difference(ImmutableSet.copyOf(HiveStorageFormat.values()), ImmutableSet.of(AVRO)); + protected Set createTableFormats = difference(ImmutableSet.copyOf(HiveStorageFormat.values()), ImmutableSet.of(AVRO, THRIFTBINARY)); private static final JoinCompiler JOIN_COMPILER = new JoinCompiler(); 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 75e296e9e0af..c75a0ddb5794 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 @@ -1867,6 +1867,9 @@ private List getAllTestingHiveStorageFormat() Session session = getSession(); ImmutableList.Builder formats = ImmutableList.builder(); for (HiveStorageFormat hiveStorageFormat : HiveStorageFormat.values()) { + if (hiveStorageFormat.equals(HiveStorageFormat.THRIFTBINARY)) { + continue; + } formats.add(new TestingHiveStorageFormat(session, hiveStorageFormat)); } formats.add(new TestingHiveStorageFormat( 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 a6fe4f6cebf9..27b85e5db4fe 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 @@ -95,6 +95,9 @@ public void testAllFormats() try { ExtendedHiveMetastore metastore = new TestingHiveMetastore(new File(tempDir, "metastore")); for (HiveStorageFormat format : HiveStorageFormat.values()) { + if (format.equals(HiveStorageFormat.THRIFTBINARY)) { + continue; + } config.setHiveStorageFormat(format); config.setHiveCompressionCodec(NONE); long uncompressedLength = writeTestFile(config, metastore, makeFileName(tempDir, config)); From 6bc1de59fcc64f735593735f384a133a26562d6e Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 28 Apr 2017 16:35:09 -0700 Subject: [PATCH 180/331] Adapt to more compact json string --- .../thrift/HiveThriftFieldIdResolver.java | 26 ++++++++++++------- .../HiveThriftFieldIdResolverFactory.java | 22 +++++++++------- 2 files changed, 29 insertions(+), 19 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java index d187bea64297..35dba2c26191 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java @@ -42,31 +42,36 @@ public short getThriftId(int hiveIndex) Short thriftId = thriftIds.get(hiveIndex); if (thriftId != null) { - return thriftId.shortValue(); + return thriftId; } else { JsonNode child = root.get(String.valueOf(hiveIndex)); checkCondition(child != null, HIVE_INVALID_METADATA, "Missed json value for hiveIndex: %s, root: %s", hiveIndex, root); - checkCondition(child.get("id") != null, HIVE_INVALID_METADATA, "Missed key id for hiveIndex: %s, root: %s", hiveIndex, root); - thriftId = Short.valueOf((short) child.get("id").asInt()); - thriftIds.put(Integer.valueOf(hiveIndex), thriftId); + if (child.isNumber()) { + thriftId = (short) child.asInt(); + } + else { + checkCondition(child.get("id") != null, HIVE_INVALID_METADATA, "Missed id for hiveIndex: %s, root: %s", hiveIndex, root); + thriftId = (short) child.get("id").asInt(); + } + thriftIds.put(hiveIndex, thriftId); return thriftId; } } public ThriftFieldIdResolver getNestedResolver(int hiveIndex) { + if (root == null) { + return this; + } + ThriftFieldIdResolver nestedResolver = nestedResolvers.get(hiveIndex); if (nestedResolver != null) { return nestedResolver; } else { - JsonNode child = null; - if (root != null) { - child = root.get(String.valueOf(hiveIndex)); - } - // what if the child == null? - // checkCondition(child != null, HIVE_INVALID_METADATA, "Missed json value for hiveIndex: %s, root: %s", hiveIndex, root); + JsonNode child = root.get(String.valueOf(hiveIndex)); + checkCondition(child != null, HIVE_INVALID_METADATA, "Missed json value for hiveIndex: %s, root: %s", hiveIndex, root); nestedResolver = new HiveThriftFieldIdResolver(child); nestedResolvers.put(hiveIndex, nestedResolver); return nestedResolver; @@ -79,6 +84,7 @@ public String toString() return toStringHelper(this) .add("root", root) .add("nestedResolvers", nestedResolvers) + .add("thriftIds", thriftIds) .toString(); } } diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolverFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolverFactory.java index 1df65fa63796..9988fdd767a8 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolverFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolverFactory.java @@ -26,19 +26,23 @@ public class HiveThriftFieldIdResolverFactory private static final Logger log = Logger.get(HiveThriftFieldIdResolverFactory.class); private static final ObjectMapper objectMapper = new ObjectMapper(); public static final String THRIFT_FIELD_ID_JSON = "thrift.field.id.json"; + public static final ThriftFieldIdResolver PLUSONE = new HiveThriftFieldIdResolver(null); public ThriftFieldIdResolver createResolver(Properties schema) { - JsonNode root = null; String jsonData = schema.getProperty(THRIFT_FIELD_ID_JSON); - if (jsonData != null) { - try { - root = objectMapper.readTree(jsonData); - } - catch (IOException e) { - log.debug(e, "Failed to createResolver, schema: %s", schema); - } + if (jsonData == null) { + return PLUSONE; } - return new HiveThriftFieldIdResolver(root); + + try { + JsonNode root = objectMapper.readTree(jsonData); + return new HiveThriftFieldIdResolver(root); + } + catch (IOException e) { + log.debug(e, "Failed to create an optimized thrift id resolver, json string: %s, schema: %s. Will use a default resolver.", jsonData, schema); + } + + return PLUSONE; } } From c3ddc7e9582d203c02565c44224acb7073ce6bc9 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 28 Apr 2017 17:58:20 -0700 Subject: [PATCH 181/331] Add tests for HiveThriftFieldIdResolver --- .../thrift/TestHiveThriftFieldIdResolver.java | 100 ++++++++++++++++++ 1 file changed, 100 insertions(+) create mode 100644 presto-hive/src/test/java/com/facebook/presto/twitter/hive/thrift/TestHiveThriftFieldIdResolver.java diff --git a/presto-hive/src/test/java/com/facebook/presto/twitter/hive/thrift/TestHiveThriftFieldIdResolver.java b/presto-hive/src/test/java/com/facebook/presto/twitter/hive/thrift/TestHiveThriftFieldIdResolver.java new file mode 100644 index 000000000000..10fa25322a94 --- /dev/null +++ b/presto-hive/src/test/java/com/facebook/presto/twitter/hive/thrift/TestHiveThriftFieldIdResolver.java @@ -0,0 +1,100 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.twitter.hive.thrift; + +import com.facebook.presto.spi.PrestoException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import org.testng.annotations.Test; + +import java.util.Map; +import java.util.Properties; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertThrows; + +@Test +public class TestHiveThriftFieldIdResolver +{ + private static final Map STRUCT_FIELD_ID_AS_MAP = ImmutableMap.of( + "0", (short) 1, + "1", (short) 2, + "id", (short) 4); + + private static final Map LIST_FIELD_ID_AS_MAP = ImmutableMap.of( + "0", "{}", + "id", (short) 5); + + private static final Map VERBOSE_PRIMARY_FIELD_ID_AS_MAP = ImmutableMap.of( + "id", (short) 6); + + private static final Map THRIFT_FIELD_ID_JSON_AS_MAP = ImmutableMap.builder() + .put("0", (short) 1) + .put("1", (short) 3) + .put("2", STRUCT_FIELD_ID_AS_MAP) + .put("3", LIST_FIELD_ID_AS_MAP) + .put("4", VERBOSE_PRIMARY_FIELD_ID_AS_MAP) + .build(); + + private final ObjectMapper objectMapper = new ObjectMapper(); + private final ThriftFieldIdResolverFactory resolverFactory = new HiveThriftFieldIdResolverFactory(); + + @Test + public void testDefaultResolver() + throws Exception + { + ThriftFieldIdResolver plusOne = resolverFactory.createResolver(new Properties()); + + assertEquals(plusOne.getThriftId(0), 1); + assertEquals(plusOne.getThriftId(10), 11); + assertEquals(plusOne.getThriftId(5), 6); + assertEquals(plusOne.getNestedResolver(2), plusOne); + assertEquals(plusOne.getNestedResolver(6), plusOne); + } + + @Test + public void testOptimizedResolver() + throws Exception + { + String json = objectMapper.writeValueAsString(THRIFT_FIELD_ID_JSON_AS_MAP); + Properties schema = new Properties(); + schema.setProperty(HiveThriftFieldIdResolverFactory.THRIFT_FIELD_ID_JSON, json); + ThriftFieldIdResolver resolver = resolverFactory.createResolver(schema); + + // primary field + assertEquals(resolver.getThriftId(0), THRIFT_FIELD_ID_JSON_AS_MAP.get("0")); + // discrete field + assertEquals(resolver.getThriftId(1), THRIFT_FIELD_ID_JSON_AS_MAP.get("1")); + + // nested field + ThriftFieldIdResolver nestedResolver = resolver.getNestedResolver(2); + Map field = (Map) THRIFT_FIELD_ID_JSON_AS_MAP.get("2"); + assertEquals(resolver.getThriftId(2), field.get("id")); + assertEquals(nestedResolver.getThriftId(0), field.get("0")); + assertEquals(nestedResolver.getThriftId(1), field.get("1")); + + // non-nested non-primary field + nestedResolver = resolver.getNestedResolver(3); + field = (Map) THRIFT_FIELD_ID_JSON_AS_MAP.get("3"); + assertEquals(resolver.getThriftId(3), field.get("id")); + + // verbose primary field + field = (Map) THRIFT_FIELD_ID_JSON_AS_MAP.get("4"); + assertEquals(resolver.getThriftId(4), field.get("id")); + + // non-existing field + assertThrows(PrestoException.class, () -> resolver.getThriftId(5)); + assertThrows(PrestoException.class, () -> resolver.getNestedResolver(5)); + } +} From 829007ce5703d0a95de48e777bbe90f5cb64204e Mon Sep 17 00:00:00 2001 From: thomass Date: Sun, 30 Apr 2017 11:06:17 -0700 Subject: [PATCH 182/331] Expose data transfer size as data flow between stages, calculated from payload field in QueryMetadata --- twitter-eventlistener-plugin/pom.xml | 5 +++++ .../QueryCompletedEventScriber.java | 18 ++++++++++++++++++ 2 files changed, 23 insertions(+) diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 1f1040072fef..10ce92d17ac9 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -30,6 +30,11 @@ com.google.guava guava + + javax.json + javax.json-api + 1.0 + diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java index 148968272af5..d95a9d9bd6a2 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java @@ -25,6 +25,11 @@ import io.airlift.log.Logger; import org.apache.thrift.TException; +import javax.json.Json; +import javax.json.JsonObject; +import javax.json.JsonReader; + +import java.io.StringReader; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -55,6 +60,12 @@ public void handle(QueryCompletedEvent event) } } + private static long getTotalDataTransferInBytes(JsonObject stage) + { + return Long.valueOf(stage.getJsonObject("stageStats").getString("outputDataSize")).longValue() + + stage.getJsonArray("subStages").stream().map(val -> (JsonObject) val).mapToLong(subStage -> getTotalDataTransferInBytes(subStage)).sum(); + } + private static QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletedEvent event) { QueryMetadata eventMetadata = event.getMetadata(); @@ -93,6 +104,13 @@ private static QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletedE thriftEvent.distributed_planning_time_ms = eventStat.getDistributedPlanningTime().get().toMillis(); } thriftEvent.total_bytes = eventStat.getTotalBytes(); + if (eventMetadata.getPayload().isPresent()) { + JsonReader jsonReader = Json.createReader(new StringReader(eventMetadata.getPayload().get())); + long totalDataTransferBytes = getTotalDataTransferInBytes(jsonReader.readObject()); + // uncomment after new thrift def is built and uploaded + //thriftEvent.total_data_transfer_bytes = totalDataTransferBytes; + jsonReader.close(); + } thriftEvent.total_rows = eventStat.getTotalRows(); thriftEvent.splits = eventStat.getCompletedSplits(); if (event.getFailureInfo().isPresent()) { From b91f46e7298d0500092d0e10a83cacad44135944 Mon Sep 17 00:00:00 2001 From: thomass Date: Sun, 30 Apr 2017 13:35:15 -0700 Subject: [PATCH 183/331] safe guard number parse error --- .../eventlistener/QueryCompletedEventScriber.java | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java index d95a9d9bd6a2..0121adeb006a 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java @@ -60,9 +60,20 @@ public void handle(QueryCompletedEvent event) } } + private static long getLongOrZero(String strVal) + { + try { + return Long.valueOf(strVal).longValue(); + } + catch (NumberFormatException e) { + log.warn(e, "Failed to parse long, returning 0"); + return 0; + } + } + private static long getTotalDataTransferInBytes(JsonObject stage) { - return Long.valueOf(stage.getJsonObject("stageStats").getString("outputDataSize")).longValue() + + return getLongOrZero(stage.getJsonObject("stageStats").getString("outputDataSize")) + stage.getJsonArray("subStages").stream().map(val -> (JsonObject) val).mapToLong(subStage -> getTotalDataTransferInBytes(subStage)).sum(); } From 082884f2df002d515106fc50100008b06ab11d02 Mon Sep 17 00:00:00 2001 From: thomass Date: Mon, 1 May 2017 23:19:21 -0700 Subject: [PATCH 184/331] Address Yaliang's comment which exposed a bug --- twitter-eventlistener-plugin/pom.xml | 5 +++++ .../plugin/eventlistener/QueryCompletedEventScriber.java | 7 ++++--- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 10ce92d17ac9..4f9873b4e7d0 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -26,6 +26,11 @@ io.airlift log + + io.airlift + units + provided + com.google.guava guava diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java index 0121adeb006a..8876bf49f164 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java @@ -23,6 +23,7 @@ import com.twitter.presto.thriftjava.QueryState; import io.airlift.log.Logger; +import io.airlift.units.DataSize; import org.apache.thrift.TException; import javax.json.Json; @@ -63,10 +64,10 @@ public void handle(QueryCompletedEvent event) private static long getLongOrZero(String strVal) { try { - return Long.valueOf(strVal).longValue(); + return DataSize.valueOf(strVal).toBytes(); } - catch (NumberFormatException e) { - log.warn(e, "Failed to parse long, returning 0"); + catch (IllegalArgumentException e) { + log.warn(e, "Failed to parse io.airlift.units.DataSize string, returning 0"); return 0; } } From 1b7addc96b618d359002da8d22545f4b04c40c47 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 2 May 2017 18:50:54 -0700 Subject: [PATCH 185/331] Use PrestoException in ThriftGeneralInputFormat --- .../com/facebook/presto/hive/BackgroundHiveSplitLoader.java | 1 - .../presto/twitter/hive/thrift/ThriftGeneralInputFormat.java | 4 +++- 2 files changed, 3 insertions(+), 2 deletions(-) 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 8abfb900f608..10f65fc22e6a 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 @@ -364,7 +364,6 @@ private void loadPartition(HivePartitionMetadata partition) // If only one bucket could match: load that one file HiveFileIterator iterator = new HiveFileIterator(path, fs, directoryLister, namenodeStats, partitionName, inputFormat, schema, partitionKeys, effectivePredicate, partition.getColumnCoercions()); - if (!buckets.isEmpty()) { int bucketCount = buckets.get(0).getBucketCount(); List list = listAndSortBucketFiles(iterator, bucketCount); diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java index 13c78d85fad0..731a4a41aeea 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.twitter.hive.thrift; +import com.facebook.presto.spi.PrestoException; import com.twitter.elephantbird.mapred.input.DeprecatedFileInputFormatWrapper; import com.twitter.elephantbird.mapreduce.input.MultiInputFormat; import com.twitter.elephantbird.mapreduce.io.BinaryWritable; @@ -28,6 +29,7 @@ import static com.facebook.presto.hive.HiveErrorCode.HIVE_INVALID_METADATA; import static com.facebook.presto.hive.HiveUtil.checkCondition; +import static java.lang.String.format; import static org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS; /** @@ -53,7 +55,7 @@ private void initialize(FileSplit split, JobConf job) throws IOException setInputFormatInstance(new MultiInputFormat(new TypeRef(thriftClass) {})); } catch (ClassNotFoundException e) { - throw new RuntimeException("Failed getting class for " + thriftClassName); + throw new PrestoException(HIVE_INVALID_METADATA, format("Failed getting class for %s", thriftClassName)); } } From 5541366b38adf3d2cea7cfb9f2d26b97af721fd0 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 2 May 2017 19:07:36 -0700 Subject: [PATCH 186/331] Unsupport write in thrift generic row --- .../twitter/hive/thrift/ThriftGenericRow.java | 77 +------------------ 1 file changed, 1 insertion(+), 76 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java index 733391ed6736..7f157ef95523 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java @@ -25,7 +25,6 @@ import org.apache.thrift.protocol.TProtocol; import org.apache.thrift.protocol.TProtocolUtil; import org.apache.thrift.protocol.TSet; -import org.apache.thrift.protocol.TStruct; import org.apache.thrift.protocol.TType; import org.apache.thrift.transport.TMemoryInputTransport; import org.apache.thrift.transport.TTransport; @@ -219,81 +218,7 @@ public void setFieldValue(Fields field, Object value) public void write(TProtocol oprot) throws TException { - oprot.writeStructBegin(new TStruct("dummy")); - values.entrySet().stream() - .filter(v -> v.getValue() != null) - .forEach(v -> writeField(v.getKey(), v.getValue(), oprot)); - oprot.writeFieldStop(); - oprot.writeStructEnd(); - } - - private void writeField(short thriftId, Object value, TProtocol oprot) - { - try { - if (value instanceof Boolean) { - oprot.writeFieldBegin(new TField("", TType.BOOL, thriftId)); - oprot.writeBool((Boolean) value); - } - else if (value instanceof Byte) { - oprot.writeFieldBegin(new TField("", TType.BYTE, thriftId)); - oprot.writeByte((Byte) value); - } - else if (value instanceof Short) { - oprot.writeFieldBegin(new TField("", TType.I16, thriftId)); - oprot.writeI16((Short) value); - } - else if (value instanceof Integer) { - oprot.writeFieldBegin(new TField("", TType.I32, thriftId)); - oprot.writeI32((Integer) value); - } - else if (value instanceof Long) { - oprot.writeFieldBegin(new TField("", TType.I64, thriftId)); - oprot.writeI64((Long) value); - } - else if (value instanceof Double) { - oprot.writeFieldBegin(new TField("", TType.DOUBLE, thriftId)); - oprot.writeDouble((Double) value); - } - else if (value instanceof String) { - oprot.writeFieldBegin(new TField("", TType.STRING, thriftId)); - oprot.writeString((String) value); - } - else if (value instanceof ThriftGenericRow) { - oprot.writeFieldBegin(new TField("", TType.STRUCT, thriftId)); - ((ThriftGenericRow) value).write(oprot); - } - else if (value instanceof List) { - oprot.writeFieldBegin(new TField("", TType.LIST, thriftId)); - writeListField((List) value, oprot); - } - else if (value instanceof Set) { - oprot.writeFieldBegin(new TField("", TType.SET, thriftId)); - writeSetField((Set) value, oprot); - } - else if (value instanceof Map) { - oprot.writeFieldBegin(new TField("", TType.MAP, thriftId)); - writeMapField((Map) value, oprot); - } - oprot.writeFieldEnd(); - } - catch (TException e) { - throw new IllegalStateException(e); - } - } - - private void writeListField(List listValue, TProtocol oprot) throws TException - { - throw new UnsupportedOperationException("writeListField is not supported."); - } - - private void writeSetField(Set setValue, TProtocol oprot) throws TException - { - throw new UnsupportedOperationException("writeSetField is not supported."); - } - - private void writeMapField(Map setValue, TProtocol oprot) throws TException - { - throw new UnsupportedOperationException("writeMapField is not supported."); + throw new UnsupportedOperationException("ThriftGenericRow.write is not supported."); } public int compareTo(ThriftGenericRow other) From a8ac0081b31cec8a5c7fd60b6e2e4b8e132ccb0d Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 3 May 2017 16:15:16 -0700 Subject: [PATCH 187/331] move getSerializationClassName to ThriftHiveRecordCursorProvider --- .../main/java/com/facebook/presto/hive/HiveUtil.java | 9 +-------- .../twitter/hive/thrift/ThriftGeneralInputFormat.java | 1 + .../hive/thrift/ThriftHiveRecordCursorProvider.java | 11 ++++++++++- 3 files changed, 12 insertions(+), 9 deletions(-) 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 5991063413c9..ccc9c9c2897a 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 @@ -121,7 +121,6 @@ import static java.util.Objects.requireNonNull; import static org.apache.hadoop.hive.common.FileUtils.unescapePathName; import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.FILE_INPUT_FORMAT; -import static org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS; import static org.apache.hadoop.hive.serde.serdeConstants.DECIMAL_TYPE_NAME; import static org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_LIB; import static org.apache.hadoop.hive.serde2.ColumnProjectionUtils.READ_ALL_COLUMNS; @@ -246,6 +245,7 @@ public static void setReadColumns(Configuration configuration, List rea return MapredParquetInputFormat.class; } + // Remove this after https://github.com/twitter/elephant-bird/pull/481 is included in a release if ("com.twitter.elephantbird.mapred.input.HiveMultiInputFormat".equals(inputFormatName)) { return ThriftGeneralInputFormat.class; } @@ -262,13 +262,6 @@ static String getInputFormatName(Properties schema) return name; } - public static String getSerializationClassName(Properties schema) - { - String name = schema.getProperty(SERIALIZATION_CLASS); - checkCondition(name != null, HIVE_INVALID_METADATA, "Table or partition is missing Hive property: %s", SERIALIZATION_CLASS); - return name; - } - public static long parseHiveDate(String value) { long millis = HIVE_DATE_PARSER.parseMillis(value); diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java index 731a4a41aeea..181a463dc25a 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java @@ -36,6 +36,7 @@ * Mirror of com.twitter.elephantbird.mapred.input.HiveMultiInputFormat allows to pass the thriftClassName * directly as a property of JobConfig. * PR for twitter/elephant-bird: https://github.com/twitter/elephant-bird/pull/481 + * Remove the class once #481 is included in a release */ @SuppressWarnings("deprecation") public class ThriftGeneralInputFormat extends DeprecatedFileInputFormatWrapper 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 c2169b4b3946..179828e265d6 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 @@ -34,11 +34,13 @@ import java.util.Properties; import java.util.Set; +import static com.facebook.presto.hive.HiveErrorCode.HIVE_INVALID_METADATA; import static com.facebook.presto.hive.HiveStorageFormat.THRIFTBINARY; +import static com.facebook.presto.hive.HiveUtil.checkCondition; import static com.facebook.presto.hive.HiveUtil.createRecordReader; import static com.facebook.presto.hive.HiveUtil.getDeserializerClassName; -import static com.facebook.presto.hive.HiveUtil.getSerializationClassName; import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS; public class ThriftHiveRecordCursorProvider implements HiveRecordCursorProvider @@ -110,4 +112,11 @@ private static void setPropertyIfUnset(Properties schema, String key, String val schema.setProperty(key, value); } } + + private static String getSerializationClassName(Properties schema) + { + String name = schema.getProperty(SERIALIZATION_CLASS); + checkCondition(name != null, HIVE_INVALID_METADATA, "Table or partition is missing Hive property: %s", SERIALIZATION_CLASS); + return name; + } } From 6df4e86891ef3c00724fe99f84762ce73dd530bf Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 3 May 2017 17:35:23 -0700 Subject: [PATCH 188/331] Better naming for default thrift id resolver --- .../HiveThriftFieldIdResolverFactory.java | 7 +++-- .../thrift/TestHiveThriftFieldIdResolver.java | 31 +++++++++++++------ 2 files changed, 25 insertions(+), 13 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolverFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolverFactory.java index 9988fdd767a8..c1c736a9f453 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolverFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolverFactory.java @@ -26,13 +26,14 @@ public class HiveThriftFieldIdResolverFactory private static final Logger log = Logger.get(HiveThriftFieldIdResolverFactory.class); private static final ObjectMapper objectMapper = new ObjectMapper(); public static final String THRIFT_FIELD_ID_JSON = "thrift.field.id.json"; - public static final ThriftFieldIdResolver PLUSONE = new HiveThriftFieldIdResolver(null); + // The default resolver which returns thrift id as hive id plus one + public static final ThriftFieldIdResolver HIVE_THRIFT_FIELD_ID_DEFAULT_RESOLVER = new HiveThriftFieldIdResolver(null); public ThriftFieldIdResolver createResolver(Properties schema) { String jsonData = schema.getProperty(THRIFT_FIELD_ID_JSON); if (jsonData == null) { - return PLUSONE; + return HIVE_THRIFT_FIELD_ID_DEFAULT_RESOLVER; } try { @@ -43,6 +44,6 @@ public ThriftFieldIdResolver createResolver(Properties schema) log.debug(e, "Failed to create an optimized thrift id resolver, json string: %s, schema: %s. Will use a default resolver.", jsonData, schema); } - return PLUSONE; + return HIVE_THRIFT_FIELD_ID_DEFAULT_RESOLVER; } } diff --git a/presto-hive/src/test/java/com/facebook/presto/twitter/hive/thrift/TestHiveThriftFieldIdResolver.java b/presto-hive/src/test/java/com/facebook/presto/twitter/hive/thrift/TestHiveThriftFieldIdResolver.java index 10fa25322a94..6342a0c088e9 100644 --- a/presto-hive/src/test/java/com/facebook/presto/twitter/hive/thrift/TestHiveThriftFieldIdResolver.java +++ b/presto-hive/src/test/java/com/facebook/presto/twitter/hive/thrift/TestHiveThriftFieldIdResolver.java @@ -27,16 +27,16 @@ @Test public class TestHiveThriftFieldIdResolver { - private static final Map STRUCT_FIELD_ID_AS_MAP = ImmutableMap.of( + private static final Map STRUCT_FIELD_ID_AS_MAP = ImmutableMap.of( "0", (short) 1, "1", (short) 2, "id", (short) 4); private static final Map LIST_FIELD_ID_AS_MAP = ImmutableMap.of( - "0", "{}", + "0", STRUCT_FIELD_ID_AS_MAP, "id", (short) 5); - private static final Map VERBOSE_PRIMARY_FIELD_ID_AS_MAP = ImmutableMap.of( + private static final Map VERBOSE_PRIMARY_FIELD_ID_AS_MAP = ImmutableMap.of( "id", (short) 6); private static final Map THRIFT_FIELD_ID_JSON_AS_MAP = ImmutableMap.builder() @@ -54,13 +54,16 @@ public class TestHiveThriftFieldIdResolver public void testDefaultResolver() throws Exception { - ThriftFieldIdResolver plusOne = resolverFactory.createResolver(new Properties()); - - assertEquals(plusOne.getThriftId(0), 1); - assertEquals(plusOne.getThriftId(10), 11); - assertEquals(plusOne.getThriftId(5), 6); - assertEquals(plusOne.getNestedResolver(2), plusOne); - assertEquals(plusOne.getNestedResolver(6), plusOne); + ThriftFieldIdResolver defaultResolver = resolverFactory.createResolver(new Properties()); + + for (int i = 0; i <= 5; ++i) { + assertEquals(defaultResolver.getThriftId(i), i + 1); + assertEquals(defaultResolver.getNestedResolver(i), defaultResolver); + } + for (int i = 5; i >= 0; --i) { + assertEquals(defaultResolver.getThriftId(i), i + 1); + assertEquals(defaultResolver.getNestedResolver(i), defaultResolver); + } } @Test @@ -89,6 +92,14 @@ public void testOptimizedResolver() field = (Map) THRIFT_FIELD_ID_JSON_AS_MAP.get("3"); assertEquals(resolver.getThriftId(3), field.get("id")); + // non-primary nested field + nestedResolver = resolver.getNestedResolver(3); + field = (Map) THRIFT_FIELD_ID_JSON_AS_MAP.get("3"); + nestedResolver = nestedResolver.getNestedResolver(0); + field = (Map) field.get("0"); + assertEquals(nestedResolver.getThriftId(0), field.get("0")); + assertEquals(nestedResolver.getThriftId(1), field.get("1")); + // verbose primary field field = (Map) THRIFT_FIELD_ID_JSON_AS_MAP.get("4"); assertEquals(resolver.getThriftId(4), field.get("id")); From 7d5685fbf71d8e784b4ccb173bae514a0fc6b347 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 4 May 2017 13:23:25 -0700 Subject: [PATCH 189/331] Added example of json property --- .../thrift/HiveThriftFieldIdResolver.java | 38 +++++++++++++++++++ .../hive/thrift/ThriftHiveRecordCursor.java | 3 +- 2 files changed, 39 insertions(+), 2 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java index 35dba2c26191..67a3088afb30 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java @@ -22,6 +22,44 @@ import static com.facebook.presto.hive.HiveUtil.checkCondition; import static com.google.common.base.MoreObjects.toStringHelper; +/** + * Resolve the translation of continuous hive ids to discontinuous thrift ids by using a json property. + * Example: + * We have the thrift definition: + * + * struct Name { + * 1: string first, + * 2: string last + * } + * struct Person { + * 1: Name name, + * 3: String phone + * } + * + * Hive table for Person: + * + * +---------+-------------+----------------------------------+-----------------+ + * | hive id | column name | type | thrift field id | + * +---------+-------------+----------------------------------+-----------------+ + * | 0 | name | struct | 1 | + * +---------+-------------+----------------------------------+-----------------+ + * | 1 | phone | string | 3 | + * +---------+-------------+----------------------------------+-----------------+ + * + * The corresponding id mapping object is: + * + * x = { + * '0': { + * '0': 1, + * '1': 2, + * }, + * '1': 3 + * } + * + * The json property is: + * + * {"0":{"0":1,"1":2},"1":3} + */ public class HiveThriftFieldIdResolver implements ThriftFieldIdResolver { diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java index ffbe348950db..3110426fc591 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java @@ -25,7 +25,6 @@ import com.facebook.presto.spi.type.Decimals; import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.TypeManager; -import com.google.common.base.Throwables; import io.airlift.log.Logger; import io.airlift.slice.Slice; import io.airlift.slice.Slices; @@ -523,7 +522,7 @@ public void close() recordReader.close(); } catch (IOException e) { - throw Throwables.propagate(e); + throw new RuntimeException("Error closing thrift record reader", e); } } From f2707aee1a6334aec08d87bdc3b57ce2d48180c4 Mon Sep 17 00:00:00 2001 From: thomass Date: Sun, 7 May 2017 17:16:51 -0700 Subject: [PATCH 190/331] prepend schema name before table name to avoid confusion --- .../presto/plugin/eventlistener/QueryCompletedEventScriber.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java index 148968272af5..dccbb1a5e88d 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java @@ -74,7 +74,7 @@ private static QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletedE thriftEvent.catalog = eventContext.getCatalog().orElse(DASH); thriftEvent.schema = eventContext.getSchema().orElse(DASH); Map> queriedColumnsByTable = new HashMap>(); - event.getIoMetadata().getInputs().forEach(input -> queriedColumnsByTable.put(input.getTable(), input.getColumns())); + event.getIoMetadata().getInputs().forEach(input -> queriedColumnsByTable.put(String.format("%s.%s", input.getSchema(), input.getTable()), input.getColumns())); thriftEvent.queried_columns_by_table = queriedColumnsByTable; thriftEvent.remote_client_address = eventContext.getRemoteClientAddress().orElse(DASH); thriftEvent.user_agent = eventContext.getUserAgent().orElse(DASH); From 5f7515b33a7445c213e3a7940c7b58595c6127c4 Mon Sep 17 00:00:00 2001 From: thomass Date: Mon, 8 May 2017 16:38:01 -0700 Subject: [PATCH 191/331] expose granular info, include operator stats --- .../QueryCompletedEventScriber.java | 27 +-- .../eventlistener/QueryStatsHelper.java | 195 ++++++++++++++++++ 2 files changed, 198 insertions(+), 24 deletions(-) create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java index 8876bf49f164..a25464e0bc05 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java @@ -20,6 +20,7 @@ import com.facebook.presto.spi.eventlistener.QueryStatistics; import com.twitter.presto.thriftjava.QueryCompletionEvent; +import com.twitter.presto.thriftjava.QueryStageInfo; import com.twitter.presto.thriftjava.QueryState; import io.airlift.log.Logger; @@ -61,23 +62,6 @@ public void handle(QueryCompletedEvent event) } } - private static long getLongOrZero(String strVal) - { - try { - return DataSize.valueOf(strVal).toBytes(); - } - catch (IllegalArgumentException e) { - log.warn(e, "Failed to parse io.airlift.units.DataSize string, returning 0"); - return 0; - } - } - - private static long getTotalDataTransferInBytes(JsonObject stage) - { - return getLongOrZero(stage.getJsonObject("stageStats").getString("outputDataSize")) + - stage.getJsonArray("subStages").stream().map(val -> (JsonObject) val).mapToLong(subStage -> getTotalDataTransferInBytes(subStage)).sum(); - } - private static QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletedEvent event) { QueryMetadata eventMetadata = event.getMetadata(); @@ -116,13 +100,8 @@ private static QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletedE thriftEvent.distributed_planning_time_ms = eventStat.getDistributedPlanningTime().get().toMillis(); } thriftEvent.total_bytes = eventStat.getTotalBytes(); - if (eventMetadata.getPayload().isPresent()) { - JsonReader jsonReader = Json.createReader(new StringReader(eventMetadata.getPayload().get())); - long totalDataTransferBytes = getTotalDataTransferInBytes(jsonReader.readObject()); - // uncomment after new thrift def is built and uploaded - //thriftEvent.total_data_transfer_bytes = totalDataTransferBytes; - jsonReader.close(); - } + thriftEvent.query_stages = QueryStatsHelper.getQueryStages(eventMetadata); + thriftEvent.operator_summaries = QueryStatsHelper.getOperatorSummaries(eventStat); thriftEvent.total_rows = eventStat.getTotalRows(); thriftEvent.splits = eventStat.getCompletedSplits(); if (event.getFailureInfo().isPresent()) { diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java new file mode 100644 index 000000000000..888303b5cb8a --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java @@ -0,0 +1,195 @@ +/* + * 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.twitter.presto.plugin.eventlistener; + +import com.facebook.presto.spi.eventlistener.QueryMetadata; +import com.facebook.presto.spi.eventlistener.QueryStatistics; + +import com.twitter.presto.thriftjava.QueryStageInfo; + +import io.airlift.log.Logger; +import io.airlift.units.DataSize; +import io.airlift.units.Duration; + +import javax.json.Json; +import javax.json.JsonObject; +import javax.json.JsonReader; + +import java.io.StringReader; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Queue; + +public class QueryStatsHelper +{ + private static final Logger log = Logger.get(QueryStatsHelper.class); + + private QueryStatsHelper() + { + throw new AssertionError(); + } + + private static long getBytesOrNegativeOne(String strVal) + { + try { + return DataSize.valueOf(strVal).toBytes(); + } + catch (IllegalArgumentException e) { + log.warn(e, + String.format("Failed to parse io.airlift.units.DataSize '%s', returning -1", strVal)); + return -1; + } + } + + private static long getMillisOrNegativeOne(String strVal) + { + try { + return Duration.valueOf(strVal).toMillis(); + } + catch (IllegalArgumentException e) { + log.warn(e, + String.format("Failed to parse io.airlift.units.Duration '%s', returning -1", strVal)); + return -1; + } + } + + private static QueryStageInfo getQueryStageInfo(int stageId, JsonObject stage) + { + QueryStageInfo stageInfo = + new com.twitter.presto.thriftjava.QueryStageInfo(); + + stageInfo.stage_id = stageId; + try { + JsonObject stageStats = stage.getJsonObject("stageStats"); + stageInfo.raw_input_data_size_bytes = getBytesOrNegativeOne(stageStats.getString("rawInputDataSize")); + stageInfo.output_data_size_bytes = getBytesOrNegativeOne(stageStats.getString("outputDataSize")); + stageInfo.completed_tasks = stageStats.getInt("completedTasks"); + stageInfo.completed_drivers = stageStats.getInt("completedDrivers"); + stageInfo.cumulative_memory = stageStats.getJsonNumber("cumulativeMemory").doubleValue(); + stageInfo.peak_memory_reservation_bytes = getBytesOrNegativeOne(stageStats.getString("peakMemoryReservation")); + stageInfo.total_scheduled_time_millis = getMillisOrNegativeOne(stageStats.getString("totalScheduledTime")); + stageInfo.total_cpu_time_millis = getMillisOrNegativeOne(stageStats.getString("totalCpuTime")); + stageInfo.total_user_time_millis = getMillisOrNegativeOne(stageStats.getString("totalUserTime")); + stageInfo.total_blocked_time_millis = getMillisOrNegativeOne(stageStats.getString("totalBlockedTime")); + } catch (Exception e) { + log.error(e, String.format("Error retrieving stage stats for stage %d", stageId)); + return null; + } + + return stageInfo; + } + + private static OperatorStats getOperatorStat(JsonObject obj) + { + OperatorStats operatorStats = + new com.twitter.presto.thriftjava.OperatorStats(); + + try { + operatorStats.pipeline_id = obj.getInt("pipelineId"); + operatorStats.operator_id = obj.getInt("operatorId"); + operatorStats.plan_node_id = obj.getString("planNodeId"); + operatorStats.operator_type = obj.getString("operatorType"); + operatorStats.total_drivers = obj.getJsonNumber("totalDrivers").longValue(); + operatorStats.add_input_calls = obj.getJsonNumber("addInputCalls").longValue(); + operatorStats.add_input_wall_millis = getMillisOrNegativeOne(obj.getString("addInputWall")); + operatorStats.add_input_cpu_millis = getMillisOrNegativeOne(obj.getString("addInputCpu")); + operatorStats.add_input_user_millis = getMillisOrNegativeOne(obj.getString("addInputUser")); + operatorStats.input_data_size_bytes = getBytesOrNegativeOne(obj.getString("inputDataSize")); + operatorStats.input_positions = obj.getJsonNumber("inputPositions").longValue(); + operatorStats.sum_squared_input_positions = obj.getJsonNumber("sumSquaredInputPositions").doubleValue(); + operatorStats.get_output_calls = obj.getJsonNumber("getOutputCalls").longValue(); + operatorStats.get_output_wall_millis = getMillisOrNegativeOne(obj.getString("getOutputWall")); + operatorStats.get_output_cpu_millis = getMillisOrNegativeOne(obj.getString("getOutputCpu")); + operatorStats.get_output_user_millis = getMillisOrNegativeOne(obj.getString("getOutputUser")); + operatorStats.output_data_size_bytes = getBytesOrNegativeOne(obj.getString("outputDataSize")); + operatorStats.output_positions = obj.getJsonNumber("outputPositions").longValue(); + operatorStats.blocked_wall_millis = getMillisOrNegativeOne(obj.getString("blockedWall")); + operatorStats.finish_calls = obj.getJsonNumber("finishCalls").longValue(); + operatorStats.finish_wall_millis = getMillisOrNegativeOne(obj.getString("finishWall")); + operatorStats.finish_cpu_millis = getMillisOrNegativeOne(obj.getString("finishCpu")); + operatorStats.finish_user_millis = getMillisOrNegativeOne(obj.getString("finishUser")); + operatorStats.memory_reservation_bytes = getBytesOrNegativeOne(obj.getString("memoryReservation")); + operatorStats.system_memory_reservation_bytes = getBytesOrNegativeOne(obj.getString("systemMemoryReservation")); + } catch (Exception e) { + log.error(e, String.format("Error retrieving operator stats from JsonObject:\n%s\n", obj.toString())); + return null; + } + + return operatorStats; + } + + public static Map getQueryStages(QueryMetadata eventMetadata) + { + if (!eventMetadata.getPayload().isPresent()) + return null; + + String payload = eventMetadata.getPayload().get(); + JsonReader jsonReader = Json.createReader(new StringReader(payload)); + Queue stageJsonObjs = new LinkedList(); + try { + stageJsonObjs.add(jsonReader.readObject()); + } catch (Exception e) { + log.warn(e, + String.format("getQueryStages - Unable to extract JsonObject out of following blob:\n%s\n", payload)); + return null; + } + + Map stages = new HashMap(); + while (!stageJsonObjs.isEmpty()) { + JsonObject cur = stageJsonObjs.poll(); + String stageIdStr = "Unknown"; + try { + stageIdStr = cur.getString("stageId"); + int stageId = Integer.parseInt(stageIdStr.split(".")[1]); + stages.put(stageId, getQueryStageInfo(stageId, cur)); + } catch (Exception e) { + log.error(e, + String.format("Failed to parse QueryStageInfo from JsonObject:\n%s\n", cur.toString())); + return null + } + + try { + cur.getJsonArray("subStages").stream().map(val -> stageJsonObjs.add((JsonObject) val)); + } catch (Exception e) { + log.error(e, + String.format("Failed to get subStages for stage %s, treating as no subStages", stageIdStr)); + } + } + + return stages; + } + + public static List getOperatorSummaries(QueryStatistics eventStat) + { + String operatorSummariesJsonStr = eventStat.getOperatorSummaries(); + if (operatorSummariesJsonStr == null || operatorSummariesJsonStr.isEmpty()) { + log.warn("No operator summary is present"); + return null; + } + + JsonReader jsonReader = Json.createReader(new StringReader(operatorSummariesJsonStr)); + List operatorSummaries = new List(); + try { + jsonReader.readArray().stream().map(val -> operatorSummaries.add(getOperatorStat((JsonObject) val))); + } catch (Exception e) { + log.error(e, + String.format("Error converting blob to List:\n%s\n", operatorSummariesJsonStr)); + return null + } + + return operatorSummaries; + } +} From d4e92c38575162a0d7d56c4b8062a68552942100 Mon Sep 17 00:00:00 2001 From: thomass Date: Mon, 8 May 2017 16:41:45 -0700 Subject: [PATCH 192/331] remove unused imports --- .../plugin/eventlistener/QueryCompletedEventScriber.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java index a25464e0bc05..5223816678a9 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java @@ -20,18 +20,11 @@ import com.facebook.presto.spi.eventlistener.QueryStatistics; import com.twitter.presto.thriftjava.QueryCompletionEvent; -import com.twitter.presto.thriftjava.QueryStageInfo; import com.twitter.presto.thriftjava.QueryState; import io.airlift.log.Logger; -import io.airlift.units.DataSize; import org.apache.thrift.TException; -import javax.json.Json; -import javax.json.JsonObject; -import javax.json.JsonReader; - -import java.io.StringReader; import java.util.HashMap; import java.util.List; import java.util.Map; From 3bb851634c5c813e18b3dd98858006940feafb84 Mon Sep 17 00:00:00 2001 From: thomass Date: Wed, 10 May 2017 14:18:58 -0700 Subject: [PATCH 193/331] Filter out nulls in collections --- twitter-eventlistener-plugin/pom.xml | 2 +- .../eventlistener/QueryStatsHelper.java | 60 ++++++++++++------- 2 files changed, 40 insertions(+), 22 deletions(-) diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 4f9873b4e7d0..5b08394333ab 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -45,7 +45,7 @@ com.twitter presto-thrift-java - 0.0.2 + 0.0.3 com.twitter diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java index 888303b5cb8a..9e526ddcabf0 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java @@ -16,6 +16,7 @@ import com.facebook.presto.spi.eventlistener.QueryMetadata; import com.facebook.presto.spi.eventlistener.QueryStatistics; +import com.twitter.presto.thriftjava.OperatorStats; import com.twitter.presto.thriftjava.QueryStageInfo; import io.airlift.log.Logger; @@ -25,6 +26,7 @@ import javax.json.Json; import javax.json.JsonObject; import javax.json.JsonReader; +import javax.json.JsonValue.ValueType; import java.io.StringReader; import java.util.HashMap; @@ -32,6 +34,7 @@ import java.util.List; import java.util.Map; import java.util.Queue; +import java.util.stream.Collectors; public class QueryStatsHelper { @@ -68,8 +71,7 @@ private static long getMillisOrNegativeOne(String strVal) private static QueryStageInfo getQueryStageInfo(int stageId, JsonObject stage) { - QueryStageInfo stageInfo = - new com.twitter.presto.thriftjava.QueryStageInfo(); + QueryStageInfo stageInfo = new QueryStageInfo(); stageInfo.stage_id = stageId; try { @@ -84,7 +86,8 @@ private static QueryStageInfo getQueryStageInfo(int stageId, JsonObject stage) stageInfo.total_cpu_time_millis = getMillisOrNegativeOne(stageStats.getString("totalCpuTime")); stageInfo.total_user_time_millis = getMillisOrNegativeOne(stageStats.getString("totalUserTime")); stageInfo.total_blocked_time_millis = getMillisOrNegativeOne(stageStats.getString("totalBlockedTime")); - } catch (Exception e) { + } + catch (Exception e) { log.error(e, String.format("Error retrieving stage stats for stage %d", stageId)); return null; } @@ -94,8 +97,7 @@ private static QueryStageInfo getQueryStageInfo(int stageId, JsonObject stage) private static OperatorStats getOperatorStat(JsonObject obj) { - OperatorStats operatorStats = - new com.twitter.presto.thriftjava.OperatorStats(); + OperatorStats operatorStats = new OperatorStats(); try { operatorStats.pipeline_id = obj.getInt("pipelineId"); @@ -123,7 +125,8 @@ private static OperatorStats getOperatorStat(JsonObject obj) operatorStats.finish_user_millis = getMillisOrNegativeOne(obj.getString("finishUser")); operatorStats.memory_reservation_bytes = getBytesOrNegativeOne(obj.getString("memoryReservation")); operatorStats.system_memory_reservation_bytes = getBytesOrNegativeOne(obj.getString("systemMemoryReservation")); - } catch (Exception e) { + } + catch (Exception e) { log.error(e, String.format("Error retrieving operator stats from JsonObject:\n%s\n", obj.toString())); return null; } @@ -131,39 +134,49 @@ private static OperatorStats getOperatorStat(JsonObject obj) return operatorStats; } - public static Map getQueryStages(QueryMetadata eventMetadata) + public static Map getQueryStages(QueryMetadata eventMetadata) { - if (!eventMetadata.getPayload().isPresent()) + if (!eventMetadata.getPayload().isPresent()) { return null; + } String payload = eventMetadata.getPayload().get(); JsonReader jsonReader = Json.createReader(new StringReader(payload)); Queue stageJsonObjs = new LinkedList(); try { stageJsonObjs.add(jsonReader.readObject()); - } catch (Exception e) { - log.warn(e, + } + catch (Exception e) { + log.error(e, String.format("getQueryStages - Unable to extract JsonObject out of following blob:\n%s\n", payload)); return null; } - Map stages = new HashMap(); + Map stages = new HashMap(); while (!stageJsonObjs.isEmpty()) { JsonObject cur = stageJsonObjs.poll(); String stageIdStr = "Unknown"; try { stageIdStr = cur.getString("stageId"); int stageId = Integer.parseInt(stageIdStr.split(".")[1]); - stages.put(stageId, getQueryStageInfo(stageId, cur)); - } catch (Exception e) { + QueryStageInfo curStage = getQueryStageInfo(stageId, cur); + if (curStage != null) { + stages.put(stageId, getQueryStageInfo(stageId, cur)); + } + } + catch (Exception e) { log.error(e, String.format("Failed to parse QueryStageInfo from JsonObject:\n%s\n", cur.toString())); - return null + return null; } try { - cur.getJsonArray("subStages").stream().map(val -> stageJsonObjs.add((JsonObject) val)); - } catch (Exception e) { + cur.getJsonArray("subStages") + .stream() + .filter(val -> val.getValueType() == ValueType.OBJECT) + .map(val -> stageJsonObjs.add((JsonObject) val)); + } + catch (Exception e) { log.error(e, String.format("Failed to get subStages for stage %s, treating as no subStages", stageIdStr)); } @@ -181,15 +194,20 @@ public static List getOperatorSummaries(QueryStatistics eventStat } JsonReader jsonReader = Json.createReader(new StringReader(operatorSummariesJsonStr)); - List operatorSummaries = new List(); try { - jsonReader.readArray().stream().map(val -> operatorSummaries.add(getOperatorStat((JsonObject) val))); - } catch (Exception e) { + return jsonReader + .readArray() + .stream() + .filter(val -> val.getValueType() == ValueType.OBJECT) + .map(val -> getOperatorStat((JsonObject) val)) + .filter(opStat -> opStat != null) + .collect(Collectors.toList()); + } + catch (Exception e) { log.error(e, String.format("Error converting blob to List:\n%s\n", operatorSummariesJsonStr)); - return null } - return operatorSummaries; + return null; } } From 3e4920bd86df8bf67ea34277176807298de80989 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 11 May 2017 13:04:05 -0700 Subject: [PATCH 194/331] Release 0.173-tw-0.34 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 48 files changed, 50 insertions(+), 50 deletions(-) diff --git a/pom.xml b/pom.xml index 80cbb43f0a05..48f70bc7d7e9 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.173-tw-0.33 + 0.173-tw-0.34 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 57f73633e6c6..037217ccecab 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 7a73340e18a3..2ec68adc5a18 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 8dabe89e1c0e..e23e108a9bdb 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 579d20650b04..0ba2b7bd4d16 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 934671a3c775..0d2f511b36fd 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 304e776e7995..37e3b42d18bf 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.173-tw-0.33 + 0.173-tw-0.34 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 72e9190b9ad3..c6b41bbde846 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 842bd187854d..bab4effde13c 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 5a173e2f14a4..22e960e657ba 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index f9be10f853f8..4b3124089872 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 97f7c3fd38ac..4ca47cea1196 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 6ba1a5a5c1f7..22b56d05176c 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 06b98fe55d83..3b6c47c41a15 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 9e6511b677f9..7904fe2b6d50 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index aca7023060a6..e48efdace6ea 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index d4f72665764d..abea3edb43c0 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 35c0556bb36a..1d129ee1779e 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 76052ec651ec..e9d914c84b27 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 5d4a560bbf6a..25180041d613 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 62b53288395b..d32f223dd369 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index cd8369eebdd2..fb982a792ea2 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 75fcfd30e054..9722d6b26632 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 44fc6ddd5d47..4c9ad8a84466 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 57f5031f4093..752c42e584d8 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-main diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 6d45014b6c74..9b0d3263a8bd 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index bf4f00ea8ba4..54f1360c46a5 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 99abe5975afe..8e41a963ddb9 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 10c0b359fa0d..226667cce65f 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 56a1b6e9566f..3283d5f90137 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index eaa09a66ccf7..3748dfb2e541 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 025cfd8abaf0..9944ee41c7e4 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 293868d3f2f5..3ab5c885f981 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index ad10c409d538..6efe47c6459f 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.173-tw-0.33 + 0.173-tw-0.34 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 2ce7343b05d2..5b16519ff507 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 0595afa11b3d..3533c47d382a 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 4d6ca18c3f68..4ccf781c1433 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 0699e33d8b67..1f4374279c09 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 470a294e8f25..ddc675834b42 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index a5e8abb07a1a..85359a9cdf94 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index ed2ee4a8b3ea..3c795c2a497f 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 023f0c10afd9..445fa6964935 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 3ce5976946fa..56f8a97b62f7 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 14c48cba3d4a..796012c85dfb 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index d7cbc61f9569..bfeed5e3dacd 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.173-tw-0.33 + 0.173-tw-0.34 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index e396c3e8b368..e4ee30981026 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 36bd77a7bbdd..d5776823acc5 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 5b08394333ab..613e94aa1bee 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173-tw-0.33 + 0.173-tw-0.34 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.173-tw-0.33 + 0.173-tw-0.34 provided From 40a7e8e02379a2556e0ec36e0c0c5e57c5ae4368 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 12 May 2017 19:24:09 -0700 Subject: [PATCH 195/331] Fix json dependency error, fix split error, move Json.createReader into try box --- twitter-eventlistener-plugin/pom.xml | 6 +++--- .../presto/plugin/eventlistener/QueryStatsHelper.java | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 613e94aa1bee..3922a281811d 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -36,9 +36,9 @@ guava - javax.json - javax.json-api - 1.0 + org.glassfish + javax.json + 1.0.4 diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java index 9e526ddcabf0..701a5cffc04f 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java @@ -141,9 +141,9 @@ public static Map getQueryStages(QueryMetadata eventMet } String payload = eventMetadata.getPayload().get(); - JsonReader jsonReader = Json.createReader(new StringReader(payload)); Queue stageJsonObjs = new LinkedList(); try { + JsonReader jsonReader = Json.createReader(new StringReader(payload)); stageJsonObjs.add(jsonReader.readObject()); } catch (Exception e) { @@ -158,7 +158,7 @@ public static Map getQueryStages(QueryMetadata eventMet String stageIdStr = "Unknown"; try { stageIdStr = cur.getString("stageId"); - int stageId = Integer.parseInt(stageIdStr.split(".")[1]); + int stageId = Integer.parseInt(stageIdStr.split("\\.")[1]); QueryStageInfo curStage = getQueryStageInfo(stageId, cur); if (curStage != null) { stages.put(stageId, getQueryStageInfo(stageId, cur)); @@ -193,8 +193,8 @@ public static List getOperatorSummaries(QueryStatistics eventStat return null; } - JsonReader jsonReader = Json.createReader(new StringReader(operatorSummariesJsonStr)); try { + JsonReader jsonReader = Json.createReader(new StringReader(operatorSummariesJsonStr)); return jsonReader .readArray() .stream() From 27297fb909145735c10319a2aa1a5c1b1244bf1c Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 17 May 2017 01:41:04 -0700 Subject: [PATCH 196/331] Add split path in the cursor error message --- .../hive/thrift/ThriftHiveRecordCursor.java | 15 ++++++++++++++- .../thrift/ThriftHiveRecordCursorProvider.java | 2 ++ 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java index 3110426fc591..122ced137231 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java @@ -29,6 +29,7 @@ import io.airlift.slice.Slice; import io.airlift.slice.Slices; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.common.type.HiveChar; import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.common.type.HiveVarchar; @@ -73,6 +74,7 @@ import static java.lang.Float.floatToRawIntBits; import static java.lang.Math.max; import static java.lang.Math.min; +import static java.lang.String.format; import static java.util.Objects.requireNonNull; class ThriftHiveRecordCursor @@ -98,6 +100,8 @@ class ThriftHiveRecordCursor private final Object[] objects; private final boolean[] nulls; + private final Path path; + private final long start; private final long totalBytes; private final DateTimeZone hiveStorageTimeZone; @@ -109,6 +113,8 @@ class ThriftHiveRecordCursor public ThriftHiveRecordCursor( RecordReader recordReader, + Path path, + long start, long totalBytes, Properties splitSchema, List columns, @@ -117,6 +123,8 @@ public ThriftHiveRecordCursor( ThriftFieldIdResolver thriftFieldIdResolver) { requireNonNull(recordReader, "recordReader is null"); + requireNonNull(path, "path is null"); + checkArgument(start >= 0, "start is negative"); checkArgument(totalBytes >= 0, "totalBytes is negative"); requireNonNull(splitSchema, "splitSchema is null"); requireNonNull(columns, "columns is null"); @@ -124,6 +132,8 @@ public ThriftHiveRecordCursor( requireNonNull(thriftFieldIdResolver, "thriftFieldIdResolver is null"); this.recordReader = recordReader; + this.path = path; + this.start = start; this.totalBytes = totalBytes; this.key = recordReader.createKey(); this.value = recordReader.createValue(); @@ -215,7 +225,10 @@ public boolean advanceNextPosition() } catch (IOException | RuntimeException e) { closeWithSuppression(this, e); - throw new PrestoException(HIVE_CURSOR_ERROR, e); + throw new PrestoException(HIVE_CURSOR_ERROR, + format("Failed to read split: %s %s:%s, total bytes: %s, completed bytes: %s", + path, start, start + totalBytes, totalBytes, completedBytes), + e); } } 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 179828e265d6..ed2cf1b9defd 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 @@ -92,6 +92,8 @@ public Optional createRecordCursor( return Optional.of(new ThriftHiveRecordCursor<>( genericRecordReader(recordReader), + path, + start, length, schema, columns, From 391dd19fa34ec8689a8bbbc99d5f8552cdec0bbe Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 18 May 2017 15:09:18 -0700 Subject: [PATCH 197/331] Move to 0.176 version --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 48 files changed, 50 insertions(+), 50 deletions(-) diff --git a/pom.xml b/pom.xml index 48f70bc7d7e9..c1cb37c5a26b 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.173-tw-0.34 + 0.176 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 037217ccecab..8be7b8a73f0a 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 2ec68adc5a18..f4e544a96500 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index e23e108a9bdb..2565d821d2c9 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 0ba2b7bd4d16..fb4d4208dd0a 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 0d2f511b36fd..84b5b15aed7f 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 37e3b42d18bf..a26ff84bb180 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.173-tw-0.34 + 0.176 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index c6b41bbde846..96b196123c94 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index bab4effde13c..3bdcc4f12fd0 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 22e960e657ba..a37e94511e12 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 4b3124089872..e0f71384fd7d 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 4ca47cea1196..e3b25eb0f733 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 22b56d05176c..99e4e52b711b 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 3b6c47c41a15..0812a70ea401 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 7904fe2b6d50..bdf5127c32f8 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index e48efdace6ea..4ed12e06db22 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index abea3edb43c0..8214d0ab4f19 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 1d129ee1779e..392f024ac4e7 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index e9d914c84b27..409273af2bb0 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 25180041d613..1645db6319f2 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index d32f223dd369..9c70d0b23770 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index fb982a792ea2..f8e35b808ab0 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 9722d6b26632..33cdcea9a1d9 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 4c9ad8a84466..9bd791c02c09 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 752c42e584d8..208d1f069bff 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-main diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 9b0d3263a8bd..097e96d932e2 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 54f1360c46a5..f09e368ced75 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 8e41a963ddb9..3479a49542d6 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 226667cce65f..1b8cfd7c2f3a 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 3283d5f90137..89a7be9fa431 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 3748dfb2e541..f7310a1c73e3 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 9944ee41c7e4..a7b80efef77e 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 3ab5c885f981..126ca0b9b73e 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 6efe47c6459f..6f1f3265aef1 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.173-tw-0.34 + 0.176 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 5b16519ff507..6b9302b73afd 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 3533c47d382a..3cf0775cd83e 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 4ccf781c1433..4999a1d1d351 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 1f4374279c09..700183a15578 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index ddc675834b42..a12e4786b2fc 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 85359a9cdf94..47bbdeeac484 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 3c795c2a497f..b57a5cd33b95 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 445fa6964935..a0c01622b2a9 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-spi diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 56f8a97b62f7..c7e52aa36bb4 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 796012c85dfb..4ccb632e1e12 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index bfeed5e3dacd..7d929fd71e0f 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.173-tw-0.34 + 0.176 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index e4ee30981026..ae24d6c80637 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index d5776823acc5..a58ad41c8d61 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 3922a281811d..07401ed758c0 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.173-tw-0.34 + 0.176 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.173-tw-0.34 + 0.176 provided From 330f47f6fe6f2c78ab4fec257a415eceb67bb428 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Sun, 21 May 2017 15:42:15 -0700 Subject: [PATCH 198/331] Fix conflict --- .../java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java index 442eabcf6b0a..8cf4a8c11157 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java @@ -167,7 +167,7 @@ public static parquet.schema.Type getParquetType(HiveColumnHandle column, Messag private static parquet.schema.Type findParquetTypeByName(HiveColumnHandle column, MessageType messageType) { String name = column.getName(); - Type type = getParquetTypeByName(name, messageType); + parquet.schema.Type type = getParquetTypeByName(name, messageType); // when a parquet field is a hive keyword we append an _ to it in hive. When doing // a name-based lookup, we need to strip it off again if we didn't get a direct match. From 963a167f6bd63067f758655821e6050ecca1f222 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 23 May 2017 19:23:10 -0700 Subject: [PATCH 199/331] put the tmp path under the default schema's folder --- .../src/main/java/com/facebook/presto/hive/HiveWriteUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java index ee7dd77ed133..22cdc382014c 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java @@ -468,7 +468,7 @@ private static boolean isDirectory(String user, HdfsEnvironment hdfsEnvironment, public static Path createTemporaryPath(String user, HdfsEnvironment hdfsEnvironment, Path targetPath) { // use a per-user temporary directory to avoid permission problems - String temporaryPrefix = "/tmp/presto-" + user; + String temporaryPrefix = "/user/hive/warehouse/.hive-staging/presto-" + user; // use relative temporary directory on ViewFS if (isViewFileSystem(user, hdfsEnvironment, targetPath)) { From e34eec08fa4f250aed63ef64290832f7d7cf6f9d Mon Sep 17 00:00:00 2001 From: thomass Date: Mon, 5 Jun 2017 13:40:09 -0700 Subject: [PATCH 200/331] Use forEach on stream to modify external collection; map doesn't work --- .../twitter/presto/plugin/eventlistener/QueryStatsHelper.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java index 701a5cffc04f..75821efbf905 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java @@ -174,7 +174,7 @@ public static Map getQueryStages(QueryMetadata eventMet cur.getJsonArray("subStages") .stream() .filter(val -> val.getValueType() == ValueType.OBJECT) - .map(val -> stageJsonObjs.add((JsonObject) val)); + .forEach(val -> stageJsonObjs.add((JsonObject) val)); } catch (Exception e) { log.error(e, From a92181e998d6de17de994a4d0c07f037214d8106 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 6 Jun 2017 15:24:53 -0700 Subject: [PATCH 201/331] use user's home --- .../src/main/java/com/facebook/presto/hive/HiveWriteUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java index 22cdc382014c..c43e71a2f8e9 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java @@ -468,7 +468,7 @@ private static boolean isDirectory(String user, HdfsEnvironment hdfsEnvironment, public static Path createTemporaryPath(String user, HdfsEnvironment hdfsEnvironment, Path targetPath) { // use a per-user temporary directory to avoid permission problems - String temporaryPrefix = "/user/hive/warehouse/.hive-staging/presto-" + user; + String temporaryPrefix = "/user/" + user + "/warehouse/.hive-staging"; // use relative temporary directory on ViewFS if (isViewFileSystem(user, hdfsEnvironment, targetPath)) { From 89493b446f8be5997d8cfbf41f6422ef9e5e0719 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Sun, 18 Jun 2017 22:41:06 -0700 Subject: [PATCH 202/331] reset the oss base version --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-cdh4/pom.xml | 2 +- presto-hive-cdh5/pom.xml | 2 +- presto-hive-hadoop1/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 49 files changed, 51 insertions(+), 51 deletions(-) diff --git a/pom.xml b/pom.xml index d81bb5ebbcad..053f8d7d3b77 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.176-tw-0.35 + 0.176 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index a337115e1273..c32cca27ea93 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 2d835e50faea..13fc9af15cb1 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index d3585850e177..2565d821d2c9 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index e6921e8f521c..fb4d4208dd0a 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 3f9bf01b75ac..84b5b15aed7f 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 960377337c65..a26ff84bb180 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.176-tw-0.35 + 0.176 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 9019a8e8beec..96b196123c94 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index a5cfdb747bfa..3bdcc4f12fd0 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 19e97e2062f4..a37e94511e12 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 639da098645f..e0f71384fd7d 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 5f061bef1c3d..e3b25eb0f733 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index fd4fbbeee0b5..99e4e52b711b 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index fa324d30b6c9..0812a70ea401 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index ac475652d285..bdf5127c32f8 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-example-http diff --git a/presto-hive-cdh4/pom.xml b/presto-hive-cdh4/pom.xml index 936ef7c10abe..4ed12e06db22 100644 --- a/presto-hive-cdh4/pom.xml +++ b/presto-hive-cdh4/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-hive-cdh4 diff --git a/presto-hive-cdh5/pom.xml b/presto-hive-cdh5/pom.xml index cc9a61a28224..8214d0ab4f19 100644 --- a/presto-hive-cdh5/pom.xml +++ b/presto-hive-cdh5/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-hive-cdh5 diff --git a/presto-hive-hadoop1/pom.xml b/presto-hive-hadoop1/pom.xml index 932aa5aa963f..392f024ac4e7 100644 --- a/presto-hive-hadoop1/pom.xml +++ b/presto-hive-hadoop1/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-hive-hadoop1 diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 403360b0efab..409273af2bb0 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 53b70dd3a92b..0ccc0282dac6 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 5cfa4540275c..9c70d0b23770 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 862682c46e6f..f8e35b808ab0 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index bc8530e82d0e..33cdcea9a1d9 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index b555a02b00f5..9bd791c02c09 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index be92958a27bc..208d1f069bff 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-main diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index e101a404b790..097e96d932e2 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 6a915f0655b3..f09e368ced75 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index c763b30b88ee..3479a49542d6 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index f91375c650c0..1b8cfd7c2f3a 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index a7394fc8f1de..89a7be9fa431 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 1f93aeb07d80..f7310a1c73e3 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 24591a7d1de6..a7b80efef77e 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 2e460481434a..126ca0b9b73e 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index f349ab620e66..1b3324d4f1ad 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.176-tw-0.35 + 0.176 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index d283f5c68518..6b9302b73afd 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index d9149096118f..3cf0775cd83e 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index e8d3ca609466..4999a1d1d351 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 9746e1d8278c..700183a15578 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index b3ca21807542..a12e4786b2fc 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 09d708c400e2..47bbdeeac484 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 3317d089c0b9..b57a5cd33b95 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index ca55ca42cdac..a0c01622b2a9 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 96eff3a77d1d..e6c0a501c351 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.176-tw-0.35 + 0.176 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 71e3643c668d..c7e52aa36bb4 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index ec8017cee51c..4ccb632e1e12 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 589ada2d095c..7d929fd71e0f 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.176-tw-0.35 + 0.176 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 95c3fef09a99..ae24d6c80637 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 9a0ed4ce7ecf..a58ad41c8d61 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index a05012a9a2b6..07401ed758c0 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.176-tw-0.35 + 0.176 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.176-tw-0.35 + 0.176 provided From 20c28163b2942949524bb3e64c39626211ba7ec2 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Sun, 18 Jun 2017 22:54:49 -0700 Subject: [PATCH 203/331] prepare release 0.179-tw-0.36 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 46 files changed, 48 insertions(+), 48 deletions(-) diff --git a/pom.xml b/pom.xml index 8a83b80cff3a..35ab6862c142 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.179 + 0.179-tw-0.36 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 9531e8ef140d..a429b78ed102 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index ef709b7b06f0..5a62826f3e01 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index ecfa30f491b7..4fc38060c34d 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 576819057c8b..b0e99e42c24f 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index cd8623865bce..dde5572e18d0 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index a40a3a25aa55..5a59fd82848e 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.179 + 0.179-tw-0.36 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index b0377c9c5564..4302c3313dcc 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 3f056fba3767..26677923e7d0 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 2ab2d150709c..6aaca8aded5a 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 9fd493d527bb..7d2de9976131 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 76beba431d05..a70dfd7fc429 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index e23a43955508..cfb1b002babd 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 58c08f0fd136..b01a12bf2df9 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 4aa2da16a866..449aaf26b8a2 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-example-http diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 6a03a31d9bdd..3356b2574997 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index f8ca83a91507..137afd1ee848 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 5805f29e658f..7717cf7aa355 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index ddc6e48b3154..a6bb4234cfe7 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 55983acd9bd7..d59c0ea4bca2 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 6146b52e2d74..d4f488fd21e9 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 7c0bd2857e4c..50d62b627dd5 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-main diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index bf4a18e8e42b..0db8f238a553 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index a8f1886fd17e..a55ff848513d 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index a4eb4792ebe9..9cafda4f787e 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 43dbbc6caa63..9682652322c6 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index e693eb954198..ac1b2eb9a068 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 623f24686649..0efeef5da3e7 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 8f19fde5c383..1a5feaed88e3 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index f6f029ec67ff..794ead1bf3c7 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 77b1d268a9dc..a4e6e93d2ee5 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.179 + 0.179-tw-0.36 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 3d08cf52b74f..aa2b0244764c 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 1571e9c70b10..2dcb1e07775b 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 548c3b618b53..e7909a0562ef 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index bfd17a4c7198..fb3eb4ab45ed 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 58057c33cf84..22c92eb8f054 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index e75105720ba9..c8b8be64cde8 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 326dd2d61690..f15592f0a3f6 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index b6863af130cb..3e032a884e8b 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index b9f99f4a171a..09028af692db 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.179 + 0.179-tw-0.36 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index c3cc98eb2e43..f089ca39b26c 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 6e0185243930..996b904a9e00 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 7f40769b0fee..2286f2c8deb1 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.179 + 0.179-tw-0.36 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 4f0c7fc76f0d..36d62494e9e6 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 2dba36a7e1e1..a9a8c2b688e0 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179 + 0.179-tw-0.36 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 07401ed758c0..ba1fbbbc0a3d 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.176 + 0.179-tw-0.36 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.176 + 0.179-tw-0.36 provided From a7f633f08ba487b1501e7a7d0327c53566dad814 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Mon, 19 Jun 2017 08:59:06 -0700 Subject: [PATCH 204/331] Fit new event --- .../presto/hive/HiveClientModule.java | 2 -- .../eventlistener/QueryStatsHelper.java | 36 +++++++++++-------- 2 files changed, 21 insertions(+), 17 deletions(-) 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 44808c8a2172..46be3f04f0f4 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 @@ -97,8 +97,6 @@ public void configure(Binder binder) Multibinder recordCursorProviderBinder = newSetBinder(binder, HiveRecordCursorProvider.class); recordCursorProviderBinder.addBinding().to(ParquetRecordCursorProvider.class).in(Scopes.SINGLETON); - recordCursorProviderBinder.addBinding().to(ColumnarTextHiveRecordCursorProvider.class).in(Scopes.SINGLETON); - recordCursorProviderBinder.addBinding().to(ColumnarBinaryHiveRecordCursorProvider.class).in(Scopes.SINGLETON); recordCursorProviderBinder.addBinding().to(ThriftHiveRecordCursorProvider.class).in(Scopes.SINGLETON); recordCursorProviderBinder.addBinding().to(GenericHiveRecordCursorProvider.class).in(Scopes.SINGLETON); diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java index 75821efbf905..0e9431a0e6c2 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java @@ -33,6 +33,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Queue; import java.util.stream.Collectors; @@ -95,6 +96,19 @@ private static QueryStageInfo getQueryStageInfo(int stageId, JsonObject stage) return stageInfo; } + private static OperatorStats getOperatorStat(String operatorSummaryStr) + { + try { + JsonReader jsonReader = Json.createReader(new StringReader(operatorSummaryStr)); + return getOperatorStat(jsonReader.readObject()); + } + catch (Exception e) { + log.error(e, String.format("Error retrieving operator stats from string:\n%s\n", operatorSummaryStr)); + } + + return null; + } + private static OperatorStats getOperatorStat(JsonObject obj) { OperatorStats operatorStats = new OperatorStats(); @@ -187,25 +201,17 @@ public static Map getQueryStages(QueryMetadata eventMet public static List getOperatorSummaries(QueryStatistics eventStat) { - String operatorSummariesJsonStr = eventStat.getOperatorSummaries(); - if (operatorSummariesJsonStr == null || operatorSummariesJsonStr.isEmpty()) { - log.warn("No operator summary is present"); - return null; - } - try { - JsonReader jsonReader = Json.createReader(new StringReader(operatorSummariesJsonStr)); - return jsonReader - .readArray() - .stream() - .filter(val -> val.getValueType() == ValueType.OBJECT) - .map(val -> getOperatorStat((JsonObject) val)) - .filter(opStat -> opStat != null) - .collect(Collectors.toList()); + return eventStat.getOperatorSummaries() + .stream() + .filter(val -> val != null && !val.isEmpty()) + .map(QueryStatsHelper::getOperatorStat) + .filter(Objects::nonNull) + .collect(Collectors.toList()); } catch (Exception e) { log.error(e, - String.format("Error converting blob to List:\n%s\n", operatorSummariesJsonStr)); + String.format("Error converting List to List:\n%s\n", eventStat.getOperatorSummaries().toString())); } return null; From c517f408362e5b9ec48322275d6ee964dba642ed Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 20 Jun 2017 01:54:24 -0700 Subject: [PATCH 205/331] Fit temp path change for tests --- .../com/facebook/presto/hive/HiveWriteUtils.java | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java index b057d8ef7e93..70595498adbb 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java @@ -441,6 +441,17 @@ public static boolean isViewFileSystem(String user, HdfsEnvironment hdfsEnvironm } } + public static boolean isLocalFileSystem(String user, HdfsEnvironment hdfsEnvironment, Path path) + { + try { + return hdfsEnvironment.getFileSystem(user, path) + .getClass().getName().equals("org.apache.hadoop.fs.LocalFileSystem"); + } + catch (IOException e) { + throw new PrestoException(HIVE_FILESYSTEM_ERROR, "Failed checking path: " + path, e); + } + } + private static boolean isDirectory(String user, HdfsEnvironment hdfsEnvironment, Path path) { try { @@ -456,6 +467,11 @@ public static Path createTemporaryPath(String user, HdfsEnvironment hdfsEnvironm // use a per-user temporary directory to avoid permission problems String temporaryPrefix = "/user/" + user + "/warehouse/.hive-staging"; + // use a per-user temporary directory in local fs system + if (isLocalFileSystem(user, hdfsEnvironment, targetPath)) { + temporaryPrefix = "/tmp/presto-" + user; + } + // use relative temporary directory on ViewFS if (isViewFileSystem(user, hdfsEnvironment, targetPath)) { temporaryPrefix = ".hive-staging"; From 2f85330d09f9fec0ddb39c5978bdacf292974d2d Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Mon, 26 Jun 2017 16:33:41 -0700 Subject: [PATCH 206/331] Set default value for hive statistic feature to false --- .../java/com/facebook/presto/hive/HiveSessionProperties.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 489b5262f152..9c8f6383dd31 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 @@ -116,7 +116,7 @@ public HiveSessionProperties(HiveClientConfig config) booleanSessionProperty( STATISTICS_ENABLED, "Experimental: Expose table statistics", - true, + false, false)); } From 77ded78c43c6dfe0218537f7ebde0ebfa8ae9141 Mon Sep 17 00:00:00 2001 From: Xiaoyao Qian Date: Wed, 19 Jul 2017 17:39:05 -0700 Subject: [PATCH 207/331] Thrift Decoder --- presto-record-decoder/pom.xml | 61 +++ .../presto/decoder/DecoderModule.java | 2 + .../presto/decoder/FieldValueProvider.java | 6 + .../decoder/thrift/ThriftDecoderModule.java | 32 ++ .../decoder/thrift/ThriftFieldDecoder.java | 369 ++++++++++++++++++ .../decoder/thrift/ThriftGenericRow.java | 233 +++++++++++ .../decoder/thrift/ThriftRowDecoder.java | 110 ++++++ .../decoder/thrift/TestThriftDecoder.java | 113 ++++++ .../src/test/thrift/tweep.thrift | 43 ++ 9 files changed, 969 insertions(+) create mode 100644 presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftDecoderModule.java create mode 100644 presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftFieldDecoder.java create mode 100644 presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftGenericRow.java create mode 100644 presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftRowDecoder.java create mode 100644 presto-record-decoder/src/test/java/com/facebook/presto/decoder/thrift/TestThriftDecoder.java create mode 100644 presto-record-decoder/src/test/thrift/tweep.thrift diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index e7909a0562ef..d6dcac2efffa 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -82,6 +82,19 @@ jackson-annotations + + + org.apache.thrift + libthrift + + + + + commons-lang + commons-lang + 2.5 + + org.testng @@ -94,5 +107,53 @@ testing test + + com.facebook.presto + presto-main + test + + + javax.servlet-api + javax.servlet + + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + 1.8 + 1.8 + + + + org.apache.thrift.tools + maven-thrift-plugin + 0.1.11 + + /usr/local/bin/thrift + + + + thrift-sources + generate-sources + + compile + + + + thrift-test-sources + generate-test-sources + + testCompile + + + + + + diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/DecoderModule.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/DecoderModule.java index 2947f0bf0a5e..5bfe6b6a2303 100644 --- a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/DecoderModule.java +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/DecoderModule.java @@ -17,6 +17,7 @@ import com.facebook.presto.decoder.dummy.DummyDecoderModule; import com.facebook.presto.decoder.json.JsonDecoderModule; import com.facebook.presto.decoder.raw.RawDecoderModule; +import com.facebook.presto.decoder.thrift.ThriftDecoderModule; import com.google.inject.Binder; import com.google.inject.Module; import com.google.inject.Scopes; @@ -38,6 +39,7 @@ public void configure(Binder binder) binder.install(new CsvDecoderModule()); binder.install(new JsonDecoderModule()); binder.install(new RawDecoderModule()); + binder.install(new ThriftDecoderModule()); } public static void bindRowDecoder(Binder binder, Class decoderClass) diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/FieldValueProvider.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/FieldValueProvider.java index cb5f4f37e7a9..27c8a7a8ba0e 100644 --- a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/FieldValueProvider.java +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/FieldValueProvider.java @@ -14,6 +14,7 @@ package com.facebook.presto.decoder; import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.block.Block; import io.airlift.slice.Slice; /** @@ -44,4 +45,9 @@ public Slice getSlice() } public abstract boolean isNull(); + + public Block getBlock() + { + throw new PrestoException(DecoderErrorCode.DECODER_CONVERSION_NOT_SUPPORTED, "conversion to block not supported"); + } } diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftDecoderModule.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftDecoderModule.java new file mode 100644 index 000000000000..5135b8175f05 --- /dev/null +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftDecoderModule.java @@ -0,0 +1,32 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.facebook.presto.decoder.thrift; + +import com.google.inject.Binder; +import com.google.inject.Module; + +import static com.facebook.presto.decoder.DecoderModule.bindFieldDecoder; +import static com.facebook.presto.decoder.DecoderModule.bindRowDecoder; + +public class ThriftDecoderModule implements Module +{ + @Override + public void configure(Binder binder) + { + bindRowDecoder(binder, ThriftRowDecoder.class); + + bindFieldDecoder(binder, ThriftFieldDecoder.class); + } +} diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftFieldDecoder.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftFieldDecoder.java new file mode 100644 index 000000000000..c62e38b2bb06 --- /dev/null +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftFieldDecoder.java @@ -0,0 +1,369 @@ +/* + * 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.decoder.thrift; + +import com.facebook.presto.decoder.DecoderColumnHandle; +import com.facebook.presto.decoder.FieldDecoder; +import com.facebook.presto.decoder.FieldValueProvider; +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.StandardTypes; +import com.facebook.presto.spi.type.Type; +import com.google.common.collect.ImmutableSet; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import org.joda.time.DateTimeZone; + +import java.sql.Date; +import java.sql.Timestamp; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import static com.facebook.presto.spi.type.BigintType.BIGINT; +import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; +import static com.facebook.presto.spi.type.Chars.isCharType; +import static com.facebook.presto.spi.type.Chars.trimSpacesAndTruncateToLength; +import static com.facebook.presto.spi.type.DateType.DATE; +import static com.facebook.presto.spi.type.DoubleType.DOUBLE; +import static com.facebook.presto.spi.type.IntegerType.INTEGER; +import static com.facebook.presto.spi.type.RealType.REAL; +import static com.facebook.presto.spi.type.SmallintType.SMALLINT; +import static com.facebook.presto.spi.type.TimestampType.TIMESTAMP; +import static com.facebook.presto.spi.type.TinyintType.TINYINT; +import static com.facebook.presto.spi.type.VarbinaryType.VARBINARY; +import static com.facebook.presto.spi.type.Varchars.isVarcharType; +import static com.facebook.presto.spi.type.Varchars.truncateToLength; +import static com.google.common.base.Preconditions.checkArgument; +import static io.airlift.slice.Slices.EMPTY_SLICE; +import static java.lang.Float.floatToRawIntBits; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +public class ThriftFieldDecoder + implements FieldDecoder +{ + @Override + public Set> getJavaTypes() + { + return ImmutableSet.of(boolean.class, long.class, double.class, Slice.class, Block.class); + } + + @Override + public final String getRowDecoderName() + { + return ThriftRowDecoder.NAME; + } + + @Override + public String getFieldDecoderName() + { + return FieldDecoder.DEFAULT_FIELD_DECODER_NAME; + } + + @Override + public FieldValueProvider decode(Object value, DecoderColumnHandle columnHandle) + { + requireNonNull(columnHandle, "columnHandle is null"); + return new ObjectValueProvider(value, columnHandle); + } + + @Override + public String toString() + { + return format("FieldDecoder[%s/%s]", getRowDecoderName(), getFieldDecoderName()); + } + + public static class ObjectValueProvider + extends FieldValueProvider + { + protected final Object value; + protected final DecoderColumnHandle columnHandle; + + public ObjectValueProvider(Object value, DecoderColumnHandle columnHandle) + { + this.columnHandle = requireNonNull(columnHandle, "columnHandle is null"); + this.value = value; + } + + @Override + public final boolean accept(DecoderColumnHandle columnHandle) + { + return this.columnHandle.equals(columnHandle); + } + + @Override + public final boolean isNull() + { + return value == null; + } + + @Override + public boolean getBoolean() + { + return isNull() ? false : (Boolean) value; + } + + @Override + public long getLong() + { + return isNull() ? 0L : getLongExpressedValue(value); + } + + private static long getLongExpressedValue(Object value) + { + if (value instanceof Date) { + long storageTime = ((Date) value).getTime(); + // convert date from VM current time zone to UTC + long utcMillis = storageTime + DateTimeZone.getDefault().getOffset(storageTime); + return TimeUnit.MILLISECONDS.toDays(utcMillis); + } + if (value instanceof Timestamp) { + long parsedJvmMillis = ((Timestamp) value).getTime(); + DateTimeZone jvmTimeZone = DateTimeZone.getDefault(); + long convertedMillis = jvmTimeZone.convertUTCToLocal(parsedJvmMillis); + + return convertedMillis; + } + if (value instanceof Float) { + return floatToRawIntBits(((Float) value)); + } + return ((Number) value).longValue(); + } + + @Override + public double getDouble() + { + return isNull() ? 0.0d : (Double) value; + } + + @Override + public Slice getSlice() + { + return isNull() ? EMPTY_SLICE : getSliceExpressedValue(value, columnHandle.getType()); + } + + private static Slice getSliceExpressedValue(Object value, Type type) + { + Slice sliceValue; + if (value instanceof String) { + sliceValue = Slices.utf8Slice((String) value); + } + else if (value instanceof byte[]) { + sliceValue = Slices.wrappedBuffer((byte[]) value); + } + else if (value instanceof Integer) { + sliceValue = Slices.utf8Slice(value.toString()); + } + else { + throw new IllegalStateException("unsupported string field type: " + value.getClass().getName()); + } + if (isVarcharType(type)) { + sliceValue = truncateToLength(sliceValue, type); + } + if (isCharType(type)) { + sliceValue = trimSpacesAndTruncateToLength(sliceValue, type); + } + + return sliceValue; + } + + @Override + public Block getBlock() + { + if (isNull()) { + return null; + } + + Type type = columnHandle.getType(); + return serializeObject(type, null, value); + } + + private static Block serializeObject(Type type, BlockBuilder builder, Object object) + { + if (!isStructuralType(type)) { + serializePrimitive(type, builder, object); + return null; + } + else if (isArrayType(type)) { + return serializeList(type, builder, object); + } + else if (isMapType(type)) { + return serializeMap(type, builder, object); + } + else if (isRowType(type)) { + return serializeStruct(type, builder, object); + } + throw new RuntimeException("Unknown object type: " + type); + } + + private static Block serializeList(Type type, BlockBuilder builder, Object object) + { + List list = (List) object; + if (list == null) { + requireNonNull(builder, "parent builder is null").appendNull(); + return null; + } + + List typeParameters = type.getTypeParameters(); + checkArgument(typeParameters.size() == 1, "list must have exactly 1 type parameter"); + Type elementType = typeParameters.get(0); + + BlockBuilder currentBuilder; + if (builder != null) { + currentBuilder = builder.beginBlockEntry(); + } + else { + currentBuilder = elementType.createBlockBuilder(new BlockBuilderStatus(), list.size()); + } + + for (Object element : list) { + serializeObject(elementType, currentBuilder, element); + } + + if (builder != null) { + builder.closeEntry(); + return null; + } + else { + Block resultBlock = currentBuilder.build(); + return resultBlock; + } + } + + private static Block serializeMap(Type type, BlockBuilder builder, Object object) + { + Map map = (Map) object; + if (map == null) { + requireNonNull(builder, "parent builder is null").appendNull(); + return null; + } + + List typeParameters = type.getTypeParameters(); + checkArgument(typeParameters.size() == 2, "map must have exactly 2 type parameter"); + Type keyType = typeParameters.get(0); + Type valueType = typeParameters.get(1); + + BlockBuilder currentBuilder; + if (builder != null) { + currentBuilder = builder.beginBlockEntry(); + } + else { + currentBuilder = new InterleavedBlockBuilder(typeParameters, new BlockBuilderStatus(), map.size()); + } + + for (Map.Entry entry : map.entrySet()) { + // Hive skips map entries with null keys + if (entry.getKey() != null) { + serializeObject(keyType, currentBuilder, entry.getKey()); + serializeObject(valueType, currentBuilder, entry.getValue()); + } + } + + if (builder != null) { + builder.closeEntry(); + return null; + } + else { + Block resultBlock = currentBuilder.build(); + return resultBlock; + } + } + + private static Block serializeStruct(Type type, BlockBuilder builder, Object object) + { + if (object == null) { + requireNonNull(builder, "parent builder is null").appendNull(); + return null; + } + + List typeParameters = type.getTypeParameters(); + ThriftGenericRow structData = (ThriftGenericRow) object; + BlockBuilder currentBuilder; + if (builder != null) { + currentBuilder = builder.beginBlockEntry(); + } + else { + currentBuilder = new InterleavedBlockBuilder(typeParameters, new BlockBuilderStatus(), typeParameters.size()); + } + + for (int i = 0; i < typeParameters.size(); i++) { + // TODO: Handle cases where ids are not consecutive + Object fieldValue = structData.getFieldValueForThriftId((short) (i + 1)); + serializeObject(typeParameters.get(i), currentBuilder, fieldValue); + } + + if (builder != null) { + builder.closeEntry(); + return null; + } + else { + Block resultBlock = currentBuilder.build(); + return resultBlock; + } + } + + private static void serializePrimitive(Type type, BlockBuilder builder, Object object) + { + requireNonNull(builder, "parent builder is null"); + + if (object == null) { + builder.appendNull(); + return; + } + + if (BOOLEAN.equals(type)) { + BOOLEAN.writeBoolean(builder, (Boolean) object); + } + else if (BIGINT.equals(type) || INTEGER.equals(type) || SMALLINT.equals(type) || TINYINT.equals(type) + || REAL.equals(type) || DATE.equals(type) || TIMESTAMP.equals(type)) { + type.writeLong(builder, getLongExpressedValue(object)); + } + else if (DOUBLE.equals(type)) { + DOUBLE.writeDouble(builder, ((Number) object).doubleValue()); + } + else if (isVarcharType(type) || VARBINARY.equals(type) || isCharType(type)) { + type.writeSlice(builder, getSliceExpressedValue(object, type)); + } + else { + throw new UnsupportedOperationException("Unsupported primitive type: " + type); + } + } + + public static boolean isArrayType(Type type) + { + return type.getTypeSignature().getBase().equals(StandardTypes.ARRAY); + } + + public static boolean isMapType(Type type) + { + return type.getTypeSignature().getBase().equals(StandardTypes.MAP); + } + + public static boolean isRowType(Type type) + { + return type.getTypeSignature().getBase().equals(StandardTypes.ROW); + } + + public static boolean isStructuralType(Type type) + { + String baseName = type.getTypeSignature().getBase(); + return baseName.equals(StandardTypes.MAP) || baseName.equals(StandardTypes.ARRAY) || baseName.equals(StandardTypes.ROW); + } + } +} diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftGenericRow.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftGenericRow.java new file mode 100644 index 000000000000..d4061030e108 --- /dev/null +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftGenericRow.java @@ -0,0 +1,233 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.decoder.thrift; + +import io.airlift.log.Logger; +import org.apache.commons.lang.ArrayUtils; +import org.apache.thrift.TBase; +import org.apache.thrift.TException; +import org.apache.thrift.TFieldIdEnum; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.protocol.TField; +import org.apache.thrift.protocol.TList; +import org.apache.thrift.protocol.TMap; +import org.apache.thrift.protocol.TProtocol; +import org.apache.thrift.protocol.TProtocolUtil; +import org.apache.thrift.protocol.TSet; +import org.apache.thrift.protocol.TType; +import org.apache.thrift.transport.TMemoryInputTransport; +import org.apache.thrift.transport.TTransport; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class ThriftGenericRow implements TBase +{ + private static final Logger log = Logger.get(ThriftGenericRow.class); + private final Map values = new HashMap<>(); + private byte[] buf = null; + private int off = 0; + private int len = 0; + + public ThriftGenericRow() {} + + public ThriftGenericRow(Map values) + { + this.values.putAll(values); + } + + public class Fields implements TFieldIdEnum + { + private final short thriftId; + private final String fieldName; + + Fields(short thriftId, String fieldName) + { + this.thriftId = thriftId; + this.fieldName = fieldName; + } + + public short getThriftFieldId() + { + return thriftId; + } + + public String getFieldName() + { + return fieldName; + } + } + + public void read(TProtocol iprot) throws TException + { + TTransport trans = iprot.getTransport(); + buf = trans.getBuffer(); + off = trans.getBufferPosition(); + TProtocolUtil.skip(iprot, TType.STRUCT); + len = trans.getBufferPosition() - off; + } + + public void parse() throws TException + { + parse(null); + } + + public void parse(short[] thriftIds) throws TException + { + Set idSet = thriftIds == null ? null : new HashSet(Arrays.asList(ArrayUtils.toObject(thriftIds))); + TMemoryInputTransport trans = new TMemoryInputTransport(buf, off, len); + TBinaryProtocol iprot = new TBinaryProtocol(trans); + TField field; + iprot.readStructBegin(); + while (true) { + field = iprot.readFieldBegin(); + if (field.type == TType.STOP) { + break; + } + if (idSet != null && !idSet.remove(Short.valueOf(field.id))) { + TProtocolUtil.skip(iprot, field.type); + } + else { + values.put(field.id, readElem(iprot, field.type)); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + } + + private Object readElem(TProtocol iprot, byte type) throws TException + { + switch (type) { + case TType.BOOL: + return iprot.readBool(); + case TType.BYTE: + return iprot.readByte(); + case TType.I16: + return iprot.readI16(); + case TType.ENUM: + case TType.I32: + return iprot.readI32(); + case TType.I64: + return iprot.readI64(); + case TType.DOUBLE: + return iprot.readDouble(); + case TType.STRING: + return iprot.readString(); + case TType.STRUCT: + return readStruct(iprot); + case TType.LIST: + return readList(iprot); + case TType.SET: + return readSet(iprot); + case TType.MAP: + return readMap(iprot); + default: + TProtocolUtil.skip(iprot, type); + return null; + } + } + + private Object readStruct(TProtocol iprot) throws TException + { + ThriftGenericRow elem = new ThriftGenericRow(); + elem.read(iprot); + elem.parse(); + return elem; + } + + private Object readList(TProtocol iprot) throws TException + { + TList ilist = iprot.readListBegin(); + List listValue = new ArrayList<>(); + for (int i = 0; i < ilist.size; ++i) { + listValue.add(readElem(iprot, ilist.elemType)); + } + iprot.readListEnd(); + return listValue; + } + + private Object readSet(TProtocol iprot) throws TException + { + TSet iset = iprot.readSetBegin(); + List setValue = new ArrayList<>(); + for (int i = 0; i < iset.size; ++i) { + setValue.add(readElem(iprot, iset.elemType)); + } + iprot.readSetEnd(); + return setValue; + } + + private Object readMap(TProtocol iprot) throws TException + { + TMap imap = iprot.readMapBegin(); + Map mapValue = new HashMap<>(); + for (int i = 0; i < imap.size; ++i) { + mapValue.put(readElem(iprot, imap.keyType), readElem(iprot, imap.valueType)); + } + iprot.readMapEnd(); + return mapValue; + } + + public Object getFieldValueForThriftId(short thriftId) + { + return values.get(thriftId); + } + + public ThriftGenericRow deepCopy() + { + return new ThriftGenericRow(values); + } + + public void clear() {} + + public Fields fieldForId(int fieldId) + { + return new Fields((short) fieldId, "dummy"); + } + + public Object getFieldValue(Fields field) + { + return values.get(field.thriftId); + } + + public boolean isSet(Fields field) + { + return values.containsKey(field.getThriftFieldId()); + } + + public void setFieldValue(Fields field, Object value) + { + values.put(field.getThriftFieldId(), value); + } + + public void write(TProtocol oprot) throws TException + { + throw new UnsupportedOperationException("ThriftGenericRow.write is not supported."); + } + + public Map getValues() + { + return values; + } + + public int compareTo(ThriftGenericRow other) + { + throw new UnsupportedOperationException("ThriftGenericRow.compareTo is not supported."); + } +} diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftRowDecoder.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftRowDecoder.java new file mode 100644 index 000000000000..27dbff67c1b1 --- /dev/null +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftRowDecoder.java @@ -0,0 +1,110 @@ +/* + * 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.decoder.thrift; + +import com.facebook.presto.decoder.DecoderColumnHandle; +import com.facebook.presto.decoder.FieldDecoder; +import com.facebook.presto.decoder.FieldValueProvider; +import com.facebook.presto.decoder.RowDecoder; +import com.google.common.base.Splitter; +import org.apache.thrift.TDeserializer; +import org.apache.thrift.TException; + +import javax.inject.Inject; + +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Thrift specific row decoder + */ +public class ThriftRowDecoder + implements RowDecoder +{ + public static final String NAME = "thrift"; + + @Inject + public ThriftRowDecoder() + { + } + + @Override + public String getName() + { + return NAME; + } + + @Override + public boolean decodeRow(byte[] data, + Map dataMap, + Set fieldValueProviders, + List columnHandles, + Map> fieldDecoders) + { + ThriftGenericRow row = new ThriftGenericRow(); + try { + TDeserializer deser = new TDeserializer(); + deser.deserialize(row, data); + row.parse(); + } + catch (TException e) { + return true; + } + + for (DecoderColumnHandle columnHandle : columnHandles) { + if (columnHandle.isInternal()) { + continue; + } + + @SuppressWarnings("unchecked") + FieldDecoder decoder = (FieldDecoder) fieldDecoders.get(columnHandle); + + if (decoder != null) { + Object node = locateNode(row.getValues(), columnHandle); + fieldValueProviders.add(decoder.decode(node, columnHandle)); + } + } + return false; + } + + private static Object locateNode(Map map, DecoderColumnHandle columnHandle) + { + Map currentLevel = map; + Object val = null; + + Iterator it = Splitter.on('/').omitEmptyStrings().split(columnHandle.getMapping()).iterator(); + while (it.hasNext()) { + String pathElement = it.next(); + Short key = Short.valueOf(pathElement); + val = currentLevel.get(key); + + // could be because of optional fields + if (val == null) { + return null; + } + + if (val instanceof ThriftGenericRow) { + currentLevel = ((ThriftGenericRow) val).getValues(); + } + else if (it.hasNext()) { + throw new IllegalStateException("Invalid thrift field schema"); + } + } + + return val; + } +} diff --git a/presto-record-decoder/src/test/java/com/facebook/presto/decoder/thrift/TestThriftDecoder.java b/presto-record-decoder/src/test/java/com/facebook/presto/decoder/thrift/TestThriftDecoder.java new file mode 100644 index 000000000000..c38c8be0333d --- /dev/null +++ b/presto-record-decoder/src/test/java/com/facebook/presto/decoder/thrift/TestThriftDecoder.java @@ -0,0 +1,113 @@ +/* + * 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.decoder.thrift; + +import com.facebook.presto.decoder.DecoderColumnHandle; +import com.facebook.presto.decoder.DecoderTestColumnHandle; +import com.facebook.presto.decoder.FieldDecoder; +import com.facebook.presto.decoder.FieldValueProvider; +import com.facebook.presto.decoder.thrift.tweep.Location; +import com.facebook.presto.decoder.thrift.tweep.Tweet; +import com.facebook.presto.decoder.thrift.tweep.TweetType; +import com.facebook.presto.spi.type.BigintType; +import com.facebook.presto.spi.type.BooleanType; +import com.facebook.presto.spi.type.DoubleType; +import com.facebook.presto.spi.type.IntegerType; +import com.facebook.presto.spi.type.SmallintType; +import com.facebook.presto.spi.type.TinyintType; +import com.facebook.presto.spi.type.VarbinaryType; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.transport.TMemoryBuffer; +import org.testng.annotations.Test; + +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static com.facebook.presto.decoder.util.DecoderTestUtil.checkValue; +import static com.facebook.presto.spi.type.VarcharType.createVarcharType; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; + +public class TestThriftDecoder +{ + private static final ThriftFieldDecoder DEFAULT_FIELD_DECODER = new ThriftFieldDecoder(); + + private static Map> buildMap(List columns) + { + ImmutableMap.Builder> map = ImmutableMap.builder(); + for (DecoderColumnHandle column : columns) { + map.put(column, DEFAULT_FIELD_DECODER); + } + return map.build(); + } + + @Test + public void testSimple() + throws Exception + { + Tweet tweet = new Tweet(1, "newUser", "hello world") + .setLoc(new Location(1234, 5678)) + .setAge((short) 26) + .setB((byte) 10) + .setIsDeleted(false) + .setTweetType(TweetType.REPLY) + .setFullId(1234567) + .setPic("abc".getBytes()) + .setAttr(ImmutableMap.of("a", "a")); + + ThriftRowDecoder rowDecoder = new ThriftRowDecoder(); + + // schema + DecoderTestColumnHandle col1 = new DecoderTestColumnHandle("", 1, "user_id", IntegerType.INTEGER,"1", "thrift", null, false, false, false); + DecoderTestColumnHandle col2 = new DecoderTestColumnHandle("", 2, "username", createVarcharType(100), "2", "thrift", null, false, false, false); + DecoderTestColumnHandle col3 = new DecoderTestColumnHandle("", 3, "text", createVarcharType(100), "3", "thrift", null, false, false, false); + DecoderTestColumnHandle col4 = new DecoderTestColumnHandle("", 4, "loc.latitude", DoubleType.DOUBLE, "4/1", "thrift", null, false, false, false); + DecoderTestColumnHandle col5 = new DecoderTestColumnHandle("", 5, "loc.longitude", DoubleType.DOUBLE, "4/2", "thrift", null, false, false, false); + DecoderTestColumnHandle col6 = new DecoderTestColumnHandle("", 6, "tweet_type", BigintType.BIGINT, "5", "thrift", null, false, false, false); + DecoderTestColumnHandle col7 = new DecoderTestColumnHandle("", 7, "is_deleted", BooleanType.BOOLEAN, "6", "thrift", null, false, false, false); + DecoderTestColumnHandle col8 = new DecoderTestColumnHandle("", 8, "b", TinyintType.TINYINT, "7", "thrift", null, false, false, false); + DecoderTestColumnHandle col9 = new DecoderTestColumnHandle("", 9, "age", SmallintType.SMALLINT, "8", "thrift", null, false, false, false); + DecoderTestColumnHandle col10 = new DecoderTestColumnHandle("", 10, "full_id", BigintType.BIGINT, "9", "thrift", null, false, false, false); + DecoderTestColumnHandle col11 = new DecoderTestColumnHandle("", 11, "pic", VarbinaryType.VARBINARY, "10", "thrift", null, false, false, false); + DecoderTestColumnHandle col12 = new DecoderTestColumnHandle("", 12, "language", createVarcharType(100), "16", "thrift", null, false, false, false); + + List columns = ImmutableList.of(col1, col2, col3, col4, col5, col6, col7, col8, col9, col10, col11, col12); + Set providers = new HashSet<>(); + + TMemoryBuffer transport = new TMemoryBuffer(4096); + TBinaryProtocol protocol = new TBinaryProtocol(transport); + tweet.write(protocol); + + boolean corrupt = rowDecoder.decodeRow(transport.getArray(), null, providers, columns, buildMap(columns)); + assertFalse(corrupt); + assertEquals(providers.size(), columns.size()); + + checkValue(providers, col1, 1); + checkValue(providers, col2, "newUser"); + checkValue(providers, col3, "hello world"); + checkValue(providers, col4, 1234); + checkValue(providers, col5, 5678); + checkValue(providers, col6, TweetType.REPLY.getValue()); + checkValue(providers, col7, false); + checkValue(providers, col8, 10); + checkValue(providers, col9, 26); + checkValue(providers, col10, 1234567); + checkValue(providers, col11, "abc"); + checkValue(providers, col12, "english"); + } +} diff --git a/presto-record-decoder/src/test/thrift/tweep.thrift b/presto-record-decoder/src/test/thrift/tweep.thrift new file mode 100644 index 000000000000..57bcc7c317e7 --- /dev/null +++ b/presto-record-decoder/src/test/thrift/tweep.thrift @@ -0,0 +1,43 @@ +namespace java com.facebook.presto.decoder.thrift.tweep + +enum TweetType { + TWEET, + RETWEET = 2, + DM = 0xa, + REPLY +} + +struct Location { + 1: required double latitude; + 2: required double longitude; +} + +struct Tweet { + 1: required i32 userId; + 2: required string userName; + 3: required string text; + 4: optional Location loc; + 5: optional TweetType tweetType = TweetType.TWEET; + 6: optional bool isDeleted = false; + 7: optional byte b; + 8: optional i16 age; + 9: optional i64 fullId; + 10: optional binary pic; + 11: optional map attr; + 12: optional list items; + 16: optional string language = "english"; +} + +typedef list TweetList +typedef set TweetSet + +struct TweetSearchResult { + 1: TweetList tweetList; + 2: TweetSet tweetSet; +} + +exception TwitterUnavailable { + 1: string message; +} + +const i32 MAX_RESULTS = 100; From d98b04e01eb7c431be524226df1cf3ecf0b6ea32 Mon Sep 17 00:00:00 2001 From: Xiaoyao Qian Date: Wed, 19 Jul 2017 17:40:07 -0700 Subject: [PATCH 208/331] cleanup --- presto-record-decoder/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index d6dcac2efffa..93ed6ea0b475 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -86,7 +86,6 @@ org.apache.thrift libthrift - From d8a6e9e7c9dc786b5ef1a1cf8c592b27b6104ccf Mon Sep 17 00:00:00 2001 From: thomass Date: Thu, 27 Jul 2017 10:32:59 -0700 Subject: [PATCH 209/331] avoid unnecessary version conflict during merge --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 46 files changed, 48 insertions(+), 48 deletions(-) diff --git a/pom.xml b/pom.xml index 35ab6862c142..8561b48c843a 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 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 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index a429b78ed102..1446f0ecaec5 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 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 5a62826f3e01..82e3de7781d0 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 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 4fc38060c34d..680ae3708c65 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index b0e99e42c24f..6f79af2dd5d0 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index dde5572e18d0..22315fe9bff7 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 5a59fd82848e..b89c60b5aa2d 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 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 4302c3313dcc..2dd093e29d26 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 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 26677923e7d0..f1fd62ef7814 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 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 6aaca8aded5a..3085a1e8a666 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 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 7d2de9976131..0b03d9918c44 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 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index a70dfd7fc429..ab39ab07f9f6 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 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index cfb1b002babd..18b88ebabed5 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 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index b01a12bf2df9..55e73127b5ec 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 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 449aaf26b8a2..6fc8e13966cc 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 presto-example-http diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 3356b2574997..6615ffe5325f 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 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 137afd1ee848..730a96e8b62d 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 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 7717cf7aa355..6f8c74f8eb85 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 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index a6bb4234cfe7..f0a62b340c5b 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index d59c0ea4bca2..c86c46262a38 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index d4f488fd21e9..58ef713653dd 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 50d62b627dd5..eb6e5d174153 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-main diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 0db8f238a553..93e05242b469 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index a55ff848513d..9506b29cd009 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 9cafda4f787e..0ebc1ca30716 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 9682652322c6..fedb182eaf9e 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index ac1b2eb9a068..2fb13d5fee89 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 0efeef5da3e7..afe831d4aa33 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 1a5feaed88e3..44470f8a8714 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 794ead1bf3c7..eaeff5a5b52a 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index a4e6e93d2ee5..f6d724a735cf 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.179-tw-0.36 + 0.181 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index aa2b0244764c..6cf07e83e992 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 2dcb1e07775b..3a5c990b2ad1 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index e7909a0562ef..019cc7614b24 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index fb3eb4ab45ed..72792a641114 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 22c92eb8f054..0bfd1a9c5bcc 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index c8b8be64cde8..4dd80298ada9 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index f15592f0a3f6..3c0dc98b52aa 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 3e032a884e8b..ce98e42e07da 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 09028af692db..de9ed48a4945 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.179-tw-0.36 + 0.181 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index f089ca39b26c..8fd5091f3c5b 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 996b904a9e00..296043bb8e0d 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 2286f2c8deb1..d5117357abd0 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.179-tw-0.36 + 0.181 presto-tests diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 36d62494e9e6..796bd8c6837b 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index a9a8c2b688e0..aec30ce9fac1 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index ba1fbbbc0a3d..7450c5040e4f 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.179-tw-0.36 + 0.181 provided From c8a76bbcbfd0023556795e6e57355a47e31ab3df Mon Sep 17 00:00:00 2001 From: thomass Date: Thu, 27 Jul 2017 13:25:04 -0700 Subject: [PATCH 210/331] use twitter tag 0.181-tw-0.37, update ThriftHiveRecordCursorProvider function signature --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- .../twitter/hive/thrift/ThriftHiveRecordCursorProvider.java | 1 + presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 50 files changed, 52 insertions(+), 51 deletions(-) diff --git a/pom.xml b/pom.xml index 03ec681508fb..ddaf55607f08 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.181 + 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.181 + 0.181-tw-0.37 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 1446f0ecaec5..c7b0cb709357 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index f1d2ef401da8..a34e9edf331c 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 680ae3708c65..3e312dd41255 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 6f79af2dd5d0..4c5e0f90e74b 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 6ee2347ecf8e..19b38c1c5ff9 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index b89c60b5aa2d..99942fab22ca 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.181 + 0.181-tw-0.37 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 2dd093e29d26..45cac6b08720 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index f1fd62ef7814..d3fa947ee109 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 3085a1e8a666..75e1d227acf7 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 0b03d9918c44..2a499ac16bbb 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index c8905d75b87b..59ff3b03bb86 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 6529190bc3d9..564ef54c8b3d 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 5f3c2a90297c..3382fdd23f25 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 6fc8e13966cc..601f42469d35 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-example-http diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 6615ffe5325f..f509c2b7401d 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 730a96e8b62d..4c2783422b4a 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-hive 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-jdbc/pom.xml b/presto-jdbc/pom.xml index de7c493b5d56..3012eb3044c1 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index f0a62b340c5b..94ef3bc69c7a 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index c86c46262a38..38ad83f0bf10 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 58ef713653dd..483e0b2246da 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index eb6e5d174153..357dc08934ff 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-main diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 93e05242b469..33a14399fc04 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 9506b29cd009..9d1f11240be4 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 0ebc1ca30716..29cca6c69338 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index fedb182eaf9e..1e188a5e822a 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 2fb13d5fee89..8f5fa1f55d2d 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index afe831d4aa33..27d1e819e309 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 44470f8a8714..5ad3f5f8b728 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index eaeff5a5b52a..a9ee0664bec0 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index f6d724a735cf..7d55006a1c6f 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.181 + 0.181-tw-0.37 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 6cf07e83e992..bbebb06643fa 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 3a5c990b2ad1..af113f8ae402 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 019cc7614b24..4915d72cd769 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 72792a641114..087b22d1164d 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 0bfd1a9c5bcc..81722baea1d8 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 4dd80298ada9..4e1096e3cf57 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 3c0dc98b52aa..9df27545885e 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index b536e7977c59..5367e6981c7f 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index de9ed48a4945..dd44e45bfc93 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.181 + 0.181-tw-0.37 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 8fd5091f3c5b..77c6f701c613 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 296043bb8e0d..7f5b625a7f1d 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 29a0bdbe0917..bc41260dfdc3 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.181 + 0.181-tw-0.37 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index ede302110fd1..c8b7b537ee95 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index d42b474edde3..a7d885bfaea2 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index b46e7ab9627d..bf87b07f0783 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-thrift-testing-server diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 796bd8c6837b..1f6423a66dd1 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index aec30ce9fac1..d8db6ca11a3c 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 7450c5040e4f..6dbbc9f3f689 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181 + 0.181-tw-0.37 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.181 + 0.181-tw-0.37 provided From 19365642d85d09abff914678636e49292c3a1af4 Mon Sep 17 00:00:00 2001 From: Xiaoyao Qian Date: Thu, 20 Jul 2017 11:24:02 -0700 Subject: [PATCH 211/331] clone presto-kafka as presto-kafka07 plugin --- pom.xml | 1 + presto-kafka07/pom.xml | 217 ++++++++++++ .../presto/kafka/KafkaColumnHandle.java | 224 ++++++++++++ .../facebook/presto/kafka/KafkaConnector.java | 92 +++++ .../presto/kafka/KafkaConnectorConfig.java | 171 +++++++++ .../presto/kafka/KafkaConnectorFactory.java | 96 +++++ .../presto/kafka/KafkaConnectorId.java | 53 +++ .../presto/kafka/KafkaConnectorModule.java | 92 +++++ .../facebook/presto/kafka/KafkaErrorCode.java | 42 +++ .../presto/kafka/KafkaHandleResolver.java | 89 +++++ .../kafka/KafkaInternalFieldDescription.java | 281 +++++++++++++++ .../facebook/presto/kafka/KafkaMetadata.java | 243 +++++++++++++ .../facebook/presto/kafka/KafkaPlugin.java | 47 +++ .../facebook/presto/kafka/KafkaRecordSet.java | 328 ++++++++++++++++++ .../presto/kafka/KafkaRecordSetProvider.java | 95 +++++ .../kafka/KafkaSimpleConsumerManager.java | 105 ++++++ .../com/facebook/presto/kafka/KafkaSplit.java | 147 ++++++++ .../presto/kafka/KafkaSplitManager.java | 149 ++++++++ .../kafka/KafkaTableDescriptionSupplier.java | 138 ++++++++ .../presto/kafka/KafkaTableHandle.java | 151 ++++++++ .../presto/kafka/KafkaTableLayoutHandle.java | 44 +++ .../presto/kafka/KafkaTopicDescription.java | 92 +++++ .../kafka/KafkaTopicFieldDescription.java | 159 +++++++++ .../presto/kafka/KafkaTopicFieldGroup.java | 62 ++++ .../presto/kafka/KafkaTransactionHandle.java | 22 ++ .../presto/kafka/KafkaQueryRunner.java | 142 ++++++++ .../kafka/TestKafkaConnectorConfig.java | 62 ++++ .../presto/kafka/TestKafkaDistributed.java | 49 +++ .../kafka/TestKafkaIntegrationSmokeTest.java | 49 +++ .../presto/kafka/TestKafkaPlugin.java | 46 +++ .../presto/kafka/TestManySegments.java | 117 +++++++ .../kafka/TestMinimalFunctionality.java | 143 ++++++++ .../presto/kafka/util/CodecSupplier.java | 71 ++++ .../presto/kafka/util/EmbeddedKafka.java | 175 ++++++++++ .../presto/kafka/util/EmbeddedZookeeper.java | 97 ++++++ .../presto/kafka/util/JsonEncoder.java | 44 +++ .../presto/kafka/util/KafkaLoader.java | 156 +++++++++ .../presto/kafka/util/NumberEncoder.java | 37 ++ .../presto/kafka/util/NumberPartitioner.java | 38 ++ .../facebook/presto/kafka/util/TestUtils.java | 95 +++++ .../src/test/resources/tpch/customer.json | 61 ++++ .../src/test/resources/tpch/lineitem.json | 104 ++++++ .../src/test/resources/tpch/nation.json | 41 +++ .../src/test/resources/tpch/orders.json | 67 ++++ .../src/test/resources/tpch/part.json | 66 ++++ .../src/test/resources/tpch/partsupp.json | 46 +++ .../src/test/resources/tpch/region.json | 36 ++ .../src/test/resources/tpch/supplier.json | 56 +++ presto-main/etc/config.properties | 1 + presto-server/src/main/provisio/presto.xml | 6 + 50 files changed, 4945 insertions(+) create mode 100644 presto-kafka07/pom.xml create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaColumnHandle.java create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnector.java create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorConfig.java create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorFactory.java create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorId.java create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorModule.java create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaErrorCode.java create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaHandleResolver.java create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaInternalFieldDescription.java create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaMetadata.java create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaPlugin.java create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSetProvider.java create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSimpleConsumerManager.java create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplit.java create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableDescriptionSupplier.java create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableHandle.java create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableLayoutHandle.java create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicDescription.java create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicFieldDescription.java create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicFieldGroup.java create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTransactionHandle.java create mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/KafkaQueryRunner.java create mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaConnectorConfig.java create mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaDistributed.java create mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaIntegrationSmokeTest.java create mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaPlugin.java create mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/TestManySegments.java create mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/TestMinimalFunctionality.java create mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/util/CodecSupplier.java create mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedKafka.java create mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedZookeeper.java create mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/util/JsonEncoder.java create mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/util/KafkaLoader.java create mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberEncoder.java create mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberPartitioner.java create mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/util/TestUtils.java create mode 100644 presto-kafka07/src/test/resources/tpch/customer.json create mode 100644 presto-kafka07/src/test/resources/tpch/lineitem.json create mode 100644 presto-kafka07/src/test/resources/tpch/nation.json create mode 100644 presto-kafka07/src/test/resources/tpch/orders.json create mode 100644 presto-kafka07/src/test/resources/tpch/part.json create mode 100644 presto-kafka07/src/test/resources/tpch/partsupp.json create mode 100644 presto-kafka07/src/test/resources/tpch/region.json create mode 100644 presto-kafka07/src/test/resources/tpch/supplier.json diff --git a/pom.xml b/pom.xml index ddaf55607f08..9410185326b7 100644 --- a/pom.xml +++ b/pom.xml @@ -73,6 +73,7 @@ presto-jmx presto-record-decoder presto-kafka + presto-kafka07 presto-redis presto-accumulo presto-cassandra diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml new file mode 100644 index 000000000000..ffcb91138731 --- /dev/null +++ b/presto-kafka07/pom.xml @@ -0,0 +1,217 @@ + + + 4.0.0 + + + com.facebook.presto + presto-root + 0.179-tw-0.36 + + + presto-kafka07 + Presto - Kafka Connector for ver0.7 + presto-plugin + + + ${project.parent.basedir} + + + + + io.airlift + bootstrap + + + + io.airlift + json + + + + io.airlift + log + + + + io.airlift + configuration + + + + com.facebook.presto + presto-record-decoder + + + + com.google.guava + guava + + + + com.google.inject + guice + + + + com.google.inject.extensions + guice-multibindings + + + + javax.validation + validation-api + + + + org.apache.kafka + kafka_2.10 + + + + joda-time + joda-time + + + + javax.annotation + javax.annotation-api + + + + javax.inject + javax.inject + + + + com.fasterxml.jackson.core + jackson-databind + + + + + com.facebook.presto + presto-spi + provided + + + + io.airlift + slice + provided + + + + io.airlift + units + provided + + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + + + io.airlift + log-manager + runtime + + + + + org.testng + testng + test + + + + io.airlift + testing + test + + + + com.facebook.presto + presto-main + test + + + + com.facebook.presto + presto-tpch + test + + + + com.facebook.presto + presto-client + test + + + + com.facebook.presto + presto-tests + test + + + + io.airlift.tpch + tpch + test + + + + org.apache.zookeeper + zookeeper + + 3.4.9 + test + + + + com.101tec + zkclient + test + + + + org.jetbrains + annotations + test + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + + **/TestKafkaDistributed.java + + + + + + + + + ci + + + + org.apache.maven.plugins + maven-surefire-plugin + + + + + + + + + diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaColumnHandle.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaColumnHandle.java new file mode 100644 index 000000000000..b8ec023b2401 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaColumnHandle.java @@ -0,0 +1,224 @@ +/* + * 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.kafka; + +import com.facebook.presto.decoder.DecoderColumnHandle; +import com.facebook.presto.spi.ColumnMetadata; +import com.facebook.presto.spi.type.Type; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +/** + * Kafka specific connector column handle. + */ +public final class KafkaColumnHandle + implements DecoderColumnHandle, Comparable +{ + private final String connectorId; + private final int ordinalPosition; + + /** + * Column Name + */ + private final String name; + + /** + * Column type + */ + private final Type type; + + /** + * Mapping hint for the decoder. Can be null. + */ + private final String mapping; + + /** + * Data format to use (selects the decoder). Can be null. + */ + private final String dataFormat; + + /** + * Additional format hint for the selected decoder. Selects a decoder subtype (e.g. which timestamp decoder). + */ + private final String formatHint; + + /** + * True if the key decoder should be used, false if the message decoder should be used. + */ + private final boolean keyDecoder; + + /** + * True if the column should be hidden. + */ + private final boolean hidden; + + /** + * True if the column is internal to the connector and not defined by a topic definition. + */ + private final boolean internal; + + @JsonCreator + public KafkaColumnHandle( + @JsonProperty("connectorId") String connectorId, + @JsonProperty("ordinalPosition") int ordinalPosition, + @JsonProperty("name") String name, + @JsonProperty("type") Type type, + @JsonProperty("mapping") String mapping, + @JsonProperty("dataFormat") String dataFormat, + @JsonProperty("formatHint") String formatHint, + @JsonProperty("keyDecoder") boolean keyDecoder, + @JsonProperty("hidden") boolean hidden, + @JsonProperty("internal") boolean internal) + + { + this.connectorId = requireNonNull(connectorId, "connectorId is null"); + this.ordinalPosition = ordinalPosition; + this.name = requireNonNull(name, "name is null"); + this.type = requireNonNull(type, "type is null"); + this.mapping = mapping; + this.dataFormat = dataFormat; + this.formatHint = formatHint; + this.keyDecoder = keyDecoder; + this.hidden = hidden; + this.internal = internal; + } + + @JsonProperty + public String getConnectorId() + { + return connectorId; + } + + @JsonProperty + public int getOrdinalPosition() + { + return ordinalPosition; + } + + @Override + @JsonProperty + public String getName() + { + return name; + } + + @Override + @JsonProperty + public Type getType() + { + return type; + } + + @Override + @JsonProperty + public String getMapping() + { + return mapping; + } + + @Override + @JsonProperty + public String getDataFormat() + { + return dataFormat; + } + + @Override + @JsonProperty + public String getFormatHint() + { + return formatHint; + } + + @JsonProperty + public boolean isKeyDecoder() + { + return keyDecoder; + } + + @JsonProperty + public boolean isHidden() + { + return hidden; + } + + @Override + @JsonProperty + public boolean isInternal() + { + return internal; + } + + ColumnMetadata getColumnMetadata() + { + return new ColumnMetadata(name, type, null, hidden); + } + + @Override + public int hashCode() + { + return Objects.hash(connectorId, ordinalPosition, name, type, mapping, dataFormat, formatHint, keyDecoder, hidden, internal); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + KafkaColumnHandle other = (KafkaColumnHandle) obj; + return Objects.equals(this.connectorId, other.connectorId) && + Objects.equals(this.ordinalPosition, other.ordinalPosition) && + Objects.equals(this.name, other.name) && + Objects.equals(this.type, other.type) && + Objects.equals(this.mapping, other.mapping) && + Objects.equals(this.dataFormat, other.dataFormat) && + Objects.equals(this.formatHint, other.formatHint) && + Objects.equals(this.keyDecoder, other.keyDecoder) && + Objects.equals(this.hidden, other.hidden) && + Objects.equals(this.internal, other.internal); + } + + @Override + public int compareTo(KafkaColumnHandle otherHandle) + { + return Integer.compare(this.getOrdinalPosition(), otherHandle.getOrdinalPosition()); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("connectorId", connectorId) + .add("ordinalPosition", ordinalPosition) + .add("name", name) + .add("type", type) + .add("mapping", mapping) + .add("dataFormat", dataFormat) + .add("formatHint", formatHint) + .add("keyDecoder", keyDecoder) + .add("hidden", hidden) + .add("internal", internal) + .toString(); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnector.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnector.java new file mode 100644 index 000000000000..69cfe62e8737 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnector.java @@ -0,0 +1,92 @@ +/* + * 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.kafka; + +import com.facebook.presto.spi.connector.Connector; +import com.facebook.presto.spi.connector.ConnectorMetadata; +import com.facebook.presto.spi.connector.ConnectorRecordSetProvider; +import com.facebook.presto.spi.connector.ConnectorSplitManager; +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; +import com.facebook.presto.spi.transaction.IsolationLevel; +import io.airlift.bootstrap.LifeCycleManager; +import io.airlift.log.Logger; + +import javax.inject.Inject; + +import static com.facebook.presto.spi.transaction.IsolationLevel.READ_COMMITTED; +import static com.facebook.presto.spi.transaction.IsolationLevel.checkConnectorSupports; +import static java.util.Objects.requireNonNull; + +/** + * Kafka specific implementation of the Presto Connector SPI. This is a read only connector. + */ +public class KafkaConnector + implements Connector +{ + private static final Logger log = Logger.get(KafkaConnector.class); + + private final LifeCycleManager lifeCycleManager; + private final KafkaMetadata metadata; + private final KafkaSplitManager splitManager; + private final KafkaRecordSetProvider recordSetProvider; + + @Inject + public KafkaConnector( + LifeCycleManager lifeCycleManager, + KafkaMetadata metadata, + KafkaSplitManager splitManager, + KafkaRecordSetProvider recordSetProvider) + { + this.lifeCycleManager = requireNonNull(lifeCycleManager, "lifeCycleManager is null"); + this.metadata = requireNonNull(metadata, "metadata is null"); + this.splitManager = requireNonNull(splitManager, "splitManager is null"); + this.recordSetProvider = requireNonNull(recordSetProvider, "recordSetProvider is null"); + } + + @Override + public ConnectorTransactionHandle beginTransaction(IsolationLevel isolationLevel, boolean readOnly) + { + checkConnectorSupports(READ_COMMITTED, isolationLevel); + return KafkaTransactionHandle.INSTANCE; + } + + @Override + public ConnectorMetadata getMetadata(ConnectorTransactionHandle transactionHandle) + { + return metadata; + } + + @Override + public ConnectorSplitManager getSplitManager() + { + return splitManager; + } + + @Override + public ConnectorRecordSetProvider getRecordSetProvider() + { + return recordSetProvider; + } + + @Override + public final void shutdown() + { + try { + lifeCycleManager.stop(); + } + catch (Exception e) { + log.error(e, "Error shutting down connector"); + } + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorConfig.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorConfig.java new file mode 100644 index 000000000000..ae5671f6512e --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorConfig.java @@ -0,0 +1,171 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.kafka; + +import com.facebook.presto.spi.HostAddress; +import com.google.common.base.Splitter; +import com.google.common.collect.ImmutableSet; +import io.airlift.configuration.Config; +import io.airlift.units.DataSize; +import io.airlift.units.DataSize.Unit; +import io.airlift.units.Duration; +import io.airlift.units.MinDuration; + +import javax.validation.constraints.NotNull; +import javax.validation.constraints.Size; + +import java.io.File; +import java.util.Set; + +import static com.google.common.collect.Iterables.transform; + +public class KafkaConnectorConfig +{ + private static final int KAFKA_DEFAULT_PORT = 9092; + + /** + * Seed nodes for Kafka cluster. At least one must exist. + */ + private Set nodes = ImmutableSet.of(); + + /** + * Timeout to connect to Kafka. + */ + private Duration kafkaConnectTimeout = Duration.valueOf("10s"); + + /** + * Buffer size for connecting to Kafka. + */ + private DataSize kafkaBufferSize = new DataSize(64, Unit.KILOBYTE); + + /** + * The schema name to use in the connector. + */ + private String defaultSchema = "default"; + + /** + * Set of tables known to this connector. For each table, a description file may be present in the catalog folder which describes columns for the given topic. + */ + private Set tableNames = ImmutableSet.of(); + + /** + * Folder holding the JSON description files for Kafka topics. + */ + private File tableDescriptionDir = new File("etc/kafka/"); + + /** + * Whether internal columns are shown in table metadata or not. Default is no. + */ + private boolean hideInternalColumns = true; + + @NotNull + public File getTableDescriptionDir() + { + return tableDescriptionDir; + } + + @Config("kafka.table-description-dir") + public KafkaConnectorConfig setTableDescriptionDir(File tableDescriptionDir) + { + this.tableDescriptionDir = tableDescriptionDir; + return this; + } + + @NotNull + public Set getTableNames() + { + return tableNames; + } + + @Config("kafka.table-names") + public KafkaConnectorConfig setTableNames(String tableNames) + { + this.tableNames = ImmutableSet.copyOf(Splitter.on(',').omitEmptyStrings().trimResults().split(tableNames)); + return this; + } + + @NotNull + public String getDefaultSchema() + { + return defaultSchema; + } + + @Config("kafka.default-schema") + public KafkaConnectorConfig setDefaultSchema(String defaultSchema) + { + this.defaultSchema = defaultSchema; + return this; + } + + @Size(min = 1) + public Set getNodes() + { + return nodes; + } + + @Config("kafka.nodes") + public KafkaConnectorConfig setNodes(String nodes) + { + this.nodes = (nodes == null) ? null : parseNodes(nodes); + return this; + } + + @MinDuration("1s") + public Duration getKafkaConnectTimeout() + { + return kafkaConnectTimeout; + } + + @Config("kafka.connect-timeout") + public KafkaConnectorConfig setKafkaConnectTimeout(String kafkaConnectTimeout) + { + this.kafkaConnectTimeout = Duration.valueOf(kafkaConnectTimeout); + return this; + } + + public DataSize getKafkaBufferSize() + { + return kafkaBufferSize; + } + + @Config("kafka.buffer-size") + public KafkaConnectorConfig setKafkaBufferSize(String kafkaBufferSize) + { + this.kafkaBufferSize = DataSize.valueOf(kafkaBufferSize); + return this; + } + + public boolean isHideInternalColumns() + { + return hideInternalColumns; + } + + @Config("kafka.hide-internal-columns") + public KafkaConnectorConfig setHideInternalColumns(boolean hideInternalColumns) + { + this.hideInternalColumns = hideInternalColumns; + return this; + } + + public static ImmutableSet parseNodes(String nodes) + { + Splitter splitter = Splitter.on(',').omitEmptyStrings().trimResults(); + return ImmutableSet.copyOf(transform(splitter.split(nodes), KafkaConnectorConfig::toHostAddress)); + } + + private static HostAddress toHostAddress(String value) + { + return HostAddress.fromString(value).withDefaultPort(KAFKA_DEFAULT_PORT); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorFactory.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorFactory.java new file mode 100644 index 000000000000..cd729032bdb4 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorFactory.java @@ -0,0 +1,96 @@ +/* + * 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.kafka; + +import com.facebook.presto.spi.ConnectorHandleResolver; +import com.facebook.presto.spi.NodeManager; +import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.spi.connector.Connector; +import com.facebook.presto.spi.connector.ConnectorContext; +import com.facebook.presto.spi.connector.ConnectorFactory; +import com.facebook.presto.spi.type.TypeManager; +import com.google.common.base.Throwables; +import com.google.inject.Injector; +import com.google.inject.Scopes; +import com.google.inject.TypeLiteral; +import io.airlift.bootstrap.Bootstrap; +import io.airlift.json.JsonModule; + +import java.util.Map; +import java.util.Optional; +import java.util.function.Supplier; + +import static java.util.Objects.requireNonNull; + +/** + * Creates Kafka Connectors based off connectorId and specific configuration. + */ +public class KafkaConnectorFactory + implements ConnectorFactory +{ + private final Optional>> tableDescriptionSupplier; + + KafkaConnectorFactory(Optional>> tableDescriptionSupplier) + { + this.tableDescriptionSupplier = requireNonNull(tableDescriptionSupplier, "tableDescriptionSupplier is null"); + } + + @Override + public String getName() + { + return "kafka"; + } + + @Override + public ConnectorHandleResolver getHandleResolver() + { + return new KafkaHandleResolver(); + } + + @Override + public Connector create(String connectorId, Map config, ConnectorContext context) + { + requireNonNull(connectorId, "connectorId is null"); + requireNonNull(config, "config is null"); + + try { + Bootstrap app = new Bootstrap( + new JsonModule(), + new KafkaConnectorModule(), + binder -> { + binder.bind(KafkaConnectorId.class).toInstance(new KafkaConnectorId(connectorId)); + binder.bind(TypeManager.class).toInstance(context.getTypeManager()); + binder.bind(NodeManager.class).toInstance(context.getNodeManager()); + + if (tableDescriptionSupplier.isPresent()) { + binder.bind(new TypeLiteral>>() {}).toInstance(tableDescriptionSupplier.get()); + } + else { + binder.bind(new TypeLiteral>>() {}).to(KafkaTableDescriptionSupplier.class).in(Scopes.SINGLETON); + } + } + ); + + Injector injector = app.strictConfig() + .doNotInitializeLogging() + .setRequiredConfigurationProperties(config) + .initialize(); + + return injector.getInstance(KafkaConnector.class); + } + catch (Exception e) { + throw Throwables.propagate(e); + } + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorId.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorId.java new file mode 100644 index 000000000000..3470980df073 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorId.java @@ -0,0 +1,53 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.kafka; + +import java.util.Objects; + +import static java.util.Objects.requireNonNull; + +public class KafkaConnectorId +{ + private final String connectorId; + + public KafkaConnectorId(String connectorId) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null"); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + KafkaConnectorId other = (KafkaConnectorId) obj; + return Objects.equals(this.connectorId, other.connectorId); + } + + @Override + public int hashCode() + { + return Objects.hash(connectorId); + } + + @Override + public String toString() + { + return connectorId; + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorModule.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorModule.java new file mode 100644 index 000000000000..e12e70567dc0 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorModule.java @@ -0,0 +1,92 @@ +/* + * 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.kafka; + +import com.facebook.presto.decoder.DecoderModule; +import com.facebook.presto.spi.type.Type; +import com.facebook.presto.spi.type.TypeManager; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.deser.std.FromStringDeserializer; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Scopes; +import com.google.inject.multibindings.Multibinder; + +import javax.inject.Inject; + +import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature; +import static com.google.common.base.Preconditions.checkArgument; +import static io.airlift.configuration.ConfigBinder.configBinder; +import static io.airlift.json.JsonBinder.jsonBinder; +import static io.airlift.json.JsonCodecBinder.jsonCodecBinder; +import static java.util.Objects.requireNonNull; + +/** + * Guice module for the Apache Kafka connector. + */ +public class KafkaConnectorModule + implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bind(KafkaConnector.class).in(Scopes.SINGLETON); + + binder.bind(KafkaMetadata.class).in(Scopes.SINGLETON); + binder.bind(KafkaSplitManager.class).in(Scopes.SINGLETON); + binder.bind(KafkaRecordSetProvider.class).in(Scopes.SINGLETON); + + binder.bind(KafkaSimpleConsumerManager.class).in(Scopes.SINGLETON); + + configBinder(binder).bindConfig(KafkaConnectorConfig.class); + + jsonBinder(binder).addDeserializerBinding(Type.class).to(TypeDeserializer.class); + jsonCodecBinder(binder).bindJsonCodec(KafkaTopicDescription.class); + + binder.install(new DecoderModule()); + + for (KafkaInternalFieldDescription internalFieldDescription : KafkaInternalFieldDescription.getInternalFields()) { + bindInternalColumn(binder, internalFieldDescription); + } + } + + private static void bindInternalColumn(Binder binder, KafkaInternalFieldDescription fieldDescription) + { + Multibinder fieldDescriptionBinder = Multibinder.newSetBinder(binder, KafkaInternalFieldDescription.class); + fieldDescriptionBinder.addBinding().toInstance(fieldDescription); + } + + public static final class TypeDeserializer + extends FromStringDeserializer + { + private static final long serialVersionUID = 1L; + + private final TypeManager typeManager; + + @Inject + public TypeDeserializer(TypeManager typeManager) + { + super(Type.class); + this.typeManager = requireNonNull(typeManager, "typeManager is null"); + } + + @Override + protected Type _deserialize(String value, DeserializationContext context) + { + Type type = typeManager.getType(parseTypeSignature(value)); + checkArgument(type != null, "Unknown type %s", value); + return type; + } + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaErrorCode.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaErrorCode.java new file mode 100644 index 000000000000..9338b9849211 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaErrorCode.java @@ -0,0 +1,42 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.kafka; + +import com.facebook.presto.spi.ErrorCode; +import com.facebook.presto.spi.ErrorCodeSupplier; +import com.facebook.presto.spi.ErrorType; + +import static com.facebook.presto.spi.ErrorType.EXTERNAL; + +/** + * Kafka connector specific error codes. + */ +public enum KafkaErrorCode + implements ErrorCodeSupplier +{ + KAFKA_SPLIT_ERROR(0, EXTERNAL); + + private final ErrorCode errorCode; + + KafkaErrorCode(int code, ErrorType type) + { + errorCode = new ErrorCode(code + 0x0102_0000, name(), type); + } + + @Override + public ErrorCode toErrorCode() + { + return errorCode; + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaHandleResolver.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaHandleResolver.java new file mode 100644 index 000000000000..539b08270129 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaHandleResolver.java @@ -0,0 +1,89 @@ +/* + * 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.kafka; + +import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.ConnectorHandleResolver; +import com.facebook.presto.spi.ConnectorSplit; +import com.facebook.presto.spi.ConnectorTableHandle; +import com.facebook.presto.spi.ConnectorTableLayoutHandle; +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; + +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +/** + * Kafka specific {@link com.facebook.presto.spi.ConnectorHandleResolver} implementation. + */ +public class KafkaHandleResolver + implements ConnectorHandleResolver +{ + @Override + public Class getTableHandleClass() + { + return KafkaTableHandle.class; + } + + @Override + public Class getColumnHandleClass() + { + return KafkaColumnHandle.class; + } + + @Override + public Class getSplitClass() + { + return KafkaSplit.class; + } + + @Override + public Class getTableLayoutHandleClass() + { + return KafkaTableLayoutHandle.class; + } + + @Override + public Class getTransactionHandleClass() + { + return KafkaTransactionHandle.class; + } + + static KafkaTableHandle convertTableHandle(ConnectorTableHandle tableHandle) + { + requireNonNull(tableHandle, "tableHandle is null"); + checkArgument(tableHandle instanceof KafkaTableHandle, "tableHandle is not an instance of KafkaTableHandle"); + return (KafkaTableHandle) tableHandle; + } + + static KafkaColumnHandle convertColumnHandle(ColumnHandle columnHandle) + { + requireNonNull(columnHandle, "columnHandle is null"); + checkArgument(columnHandle instanceof KafkaColumnHandle, "columnHandle is not an instance of KafkaColumnHandle"); + return (KafkaColumnHandle) columnHandle; + } + + static KafkaSplit convertSplit(ConnectorSplit split) + { + requireNonNull(split, "split is null"); + checkArgument(split instanceof KafkaSplit, "split is not an instance of KafkaSplit"); + return (KafkaSplit) split; + } + + static KafkaTableLayoutHandle convertLayout(ConnectorTableLayoutHandle layout) + { + requireNonNull(layout, "layout is null"); + checkArgument(layout instanceof KafkaTableLayoutHandle, "layout is not an instance of KafkaTableLayoutHandle"); + return (KafkaTableLayoutHandle) layout; + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaInternalFieldDescription.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaInternalFieldDescription.java new file mode 100644 index 000000000000..2aa81ed6f9e3 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaInternalFieldDescription.java @@ -0,0 +1,281 @@ +/* + * 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.kafka; + +import com.facebook.presto.decoder.DecoderColumnHandle; +import com.facebook.presto.decoder.FieldValueProvider; +import com.facebook.presto.spi.ColumnMetadata; +import com.facebook.presto.spi.type.BigintType; +import com.facebook.presto.spi.type.BooleanType; +import com.facebook.presto.spi.type.Type; +import com.google.common.collect.ImmutableSet; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; + +import java.util.Objects; +import java.util.Set; + +import static com.facebook.presto.spi.type.VarcharType.createUnboundedVarcharType; +import static com.google.common.base.MoreObjects.toStringHelper; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Strings.isNullOrEmpty; +import static java.util.Objects.requireNonNull; + +/** + * Describes an internal (managed by the connector) field which is added to each table row. The definition itself makes the row + * show up in the tables (the columns are hidden by default, so they must be explicitly selected) but unless the field is hooked in using the + * forBooleanValue/forLongValue/forBytesValue methods and the resulting FieldValueProvider is then passed into the appropriate row decoder, the fields + * will be null. Most values are assigned in the {@link com.facebook.presto.kafka.KafkaRecordSet}. + */ +public class KafkaInternalFieldDescription +{ + /** + * _partition_id - Kafka partition id. + */ + public static final KafkaInternalFieldDescription PARTITION_ID_FIELD = new KafkaInternalFieldDescription("_partition_id", BigintType.BIGINT, "Partition Id"); + + /** + * _partition_offset - The current offset of the message in the partition. + */ + public static final KafkaInternalFieldDescription PARTITION_OFFSET_FIELD = new KafkaInternalFieldDescription("_partition_offset", BigintType.BIGINT, "Offset for the message within the partition"); + + /** + * _segment_start - Kafka start offset for the segment which contains the current message. This is per-partition. + */ + public static final KafkaInternalFieldDescription SEGMENT_START_FIELD = new KafkaInternalFieldDescription("_segment_start", BigintType.BIGINT, "Segment start offset"); + + /** + * _segment_end - Kafka end offset for the segment which contains the current message. This is per-partition. The end offset is the first offset that is *not* in the segment. + */ + public static final KafkaInternalFieldDescription SEGMENT_END_FIELD = new KafkaInternalFieldDescription("_segment_end", BigintType.BIGINT, "Segment end offset"); + + /** + * _segment_count - Running count of messages in a segment. + */ + public static final KafkaInternalFieldDescription SEGMENT_COUNT_FIELD = new KafkaInternalFieldDescription("_segment_count", BigintType.BIGINT, "Running message count per segment"); + + /** + * _message_corrupt - True if the row converter could not read the a message. May be null if the row converter does not set a value (e.g. the dummy row converter does not). + */ + public static final KafkaInternalFieldDescription MESSAGE_CORRUPT_FIELD = new KafkaInternalFieldDescription("_message_corrupt", BooleanType.BOOLEAN, "Message data is corrupt"); + + /** + * _message - Represents the full topic as a text column. Format is UTF-8 which may be wrong for some topics. TODO: make charset configurable. + */ + public static final KafkaInternalFieldDescription MESSAGE_FIELD = new KafkaInternalFieldDescription("_message", createUnboundedVarcharType(), "Message text"); + + /** + * _message_length - length in bytes of the message. + */ + public static final KafkaInternalFieldDescription MESSAGE_LENGTH_FIELD = new KafkaInternalFieldDescription("_message_length", BigintType.BIGINT, "Total number of message bytes"); + + /** + * _key_corrupt - True if the row converter could not read the a key. May be null if the row converter does not set a value (e.g. the dummy row converter does not). + */ + public static final KafkaInternalFieldDescription KEY_CORRUPT_FIELD = new KafkaInternalFieldDescription("_key_corrupt", BooleanType.BOOLEAN, "Key data is corrupt"); + + /** + * _key - Represents the key as a text column. Format is UTF-8 which may be wrong for topics. TODO: make charset configurable. + */ + public static final KafkaInternalFieldDescription KEY_FIELD = new KafkaInternalFieldDescription("_key", createUnboundedVarcharType(), "Key text"); + + /** + * _key_length - length in bytes of the key. + */ + public static final KafkaInternalFieldDescription KEY_LENGTH_FIELD = new KafkaInternalFieldDescription("_key_length", BigintType.BIGINT, "Total number of key bytes"); + + public static Set getInternalFields() + { + return ImmutableSet.of(PARTITION_ID_FIELD, PARTITION_OFFSET_FIELD, + SEGMENT_START_FIELD, SEGMENT_END_FIELD, SEGMENT_COUNT_FIELD, + KEY_FIELD, KEY_CORRUPT_FIELD, KEY_LENGTH_FIELD, + MESSAGE_FIELD, MESSAGE_CORRUPT_FIELD, MESSAGE_LENGTH_FIELD); + } + + private final String name; + private final Type type; + private final String comment; + + KafkaInternalFieldDescription( + String name, + Type type, + String comment) + { + checkArgument(!isNullOrEmpty(name), "name is null or is empty"); + this.name = name; + this.type = requireNonNull(type, "type is null"); + this.comment = requireNonNull(comment, "comment is null"); + } + + public String getName() + { + return name; + } + + public Type getType() + { + return type; + } + + KafkaColumnHandle getColumnHandle(String connectorId, int index, boolean hidden) + { + return new KafkaColumnHandle(connectorId, + index, + getName(), + getType(), + null, + null, + null, + false, + hidden, + true); + } + + ColumnMetadata getColumnMetadata(boolean hidden) + { + return new ColumnMetadata(name, type, comment, hidden); + } + + public FieldValueProvider forBooleanValue(boolean value) + { + return new BooleanKafkaFieldValueProvider(value); + } + + public FieldValueProvider forLongValue(long value) + { + return new LongKafkaFieldValueProvider(value); + } + + public FieldValueProvider forByteValue(byte[] value) + { + return new BytesKafkaFieldValueProvider(value); + } + + @Override + public int hashCode() + { + return Objects.hash(name, type); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + KafkaInternalFieldDescription other = (KafkaInternalFieldDescription) obj; + return Objects.equals(this.name, other.name) && + Objects.equals(this.type, other.type); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("name", name) + .add("type", type) + .toString(); + } + + public class BooleanKafkaFieldValueProvider + extends FieldValueProvider + { + private final boolean value; + + private BooleanKafkaFieldValueProvider(boolean value) + { + this.value = value; + } + + @Override + public boolean accept(DecoderColumnHandle columnHandle) + { + return columnHandle.getName().equals(name); + } + + @Override + public boolean getBoolean() + { + return value; + } + + @Override + public boolean isNull() + { + return false; + } + } + + public class LongKafkaFieldValueProvider + extends FieldValueProvider + { + private final long value; + + private LongKafkaFieldValueProvider(long value) + { + this.value = value; + } + + @Override + public boolean accept(DecoderColumnHandle columnHandle) + { + return columnHandle.getName().equals(name); + } + + @Override + public long getLong() + { + return value; + } + + @Override + public boolean isNull() + { + return false; + } + } + + public class BytesKafkaFieldValueProvider + extends FieldValueProvider + { + private final byte[] value; + + private BytesKafkaFieldValueProvider(byte[] value) + { + this.value = value; + } + + @Override + public boolean accept(DecoderColumnHandle columnHandle) + { + return columnHandle.getName().equals(name); + } + + @Override + public Slice getSlice() + { + return isNull() ? Slices.EMPTY_SLICE : Slices.wrappedBuffer(value); + } + + @Override + public boolean isNull() + { + return value == null || value.length == 0; + } + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaMetadata.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaMetadata.java new file mode 100644 index 000000000000..1815646cbda1 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaMetadata.java @@ -0,0 +1,243 @@ +/* + * 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.kafka; + +import com.facebook.presto.decoder.dummy.DummyRowDecoder; +import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.ColumnMetadata; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.ConnectorTableHandle; +import com.facebook.presto.spi.ConnectorTableLayout; +import com.facebook.presto.spi.ConnectorTableLayoutHandle; +import com.facebook.presto.spi.ConnectorTableLayoutResult; +import com.facebook.presto.spi.ConnectorTableMetadata; +import com.facebook.presto.spi.Constraint; +import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.spi.SchemaTablePrefix; +import com.facebook.presto.spi.TableNotFoundException; +import com.facebook.presto.spi.connector.ConnectorMetadata; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; + +import javax.inject.Inject; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.function.Supplier; + +import static com.facebook.presto.kafka.KafkaHandleResolver.convertColumnHandle; +import static com.facebook.presto.kafka.KafkaHandleResolver.convertTableHandle; +import static java.util.Objects.requireNonNull; + +/** + * Manages the Kafka connector specific metadata information. The Connector provides an additional set of columns + * for each table that are created as hidden columns. See {@link KafkaInternalFieldDescription} for a list + * of per-topic additional columns. + */ +public class KafkaMetadata + implements ConnectorMetadata +{ + private final String connectorId; + private final boolean hideInternalColumns; + private final Map tableDescriptions; + private final Set internalFieldDescriptions; + + @Inject + public KafkaMetadata( + KafkaConnectorId connectorId, + KafkaConnectorConfig kafkaConnectorConfig, + Supplier> kafkaTableDescriptionSupplier, + Set internalFieldDescriptions) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); + + requireNonNull(kafkaConnectorConfig, "kafkaConfig is null"); + this.hideInternalColumns = kafkaConnectorConfig.isHideInternalColumns(); + + requireNonNull(kafkaTableDescriptionSupplier, "kafkaTableDescriptionSupplier is null"); + this.tableDescriptions = kafkaTableDescriptionSupplier.get(); + this.internalFieldDescriptions = requireNonNull(internalFieldDescriptions, "internalFieldDescriptions is null"); + } + + @Override + public List listSchemaNames(ConnectorSession session) + { + ImmutableSet.Builder builder = ImmutableSet.builder(); + for (SchemaTableName tableName : tableDescriptions.keySet()) { + builder.add(tableName.getSchemaName()); + } + return ImmutableList.copyOf(builder.build()); + } + + @Override + public KafkaTableHandle getTableHandle(ConnectorSession session, SchemaTableName schemaTableName) + { + KafkaTopicDescription table = tableDescriptions.get(schemaTableName); + if (table == null) { + return null; + } + + return new KafkaTableHandle(connectorId, + schemaTableName.getSchemaName(), + schemaTableName.getTableName(), + table.getTopicName(), + getDataFormat(table.getKey()), + getDataFormat(table.getMessage())); + } + + private static String getDataFormat(KafkaTopicFieldGroup fieldGroup) + { + return (fieldGroup == null) ? DummyRowDecoder.NAME : fieldGroup.getDataFormat(); + } + + @Override + public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle tableHandle) + { + return getTableMetadata(convertTableHandle(tableHandle).toSchemaTableName()); + } + + @Override + public List listTables(ConnectorSession session, String schemaNameOrNull) + { + ImmutableList.Builder builder = ImmutableList.builder(); + for (SchemaTableName tableName : tableDescriptions.keySet()) { + if (schemaNameOrNull == null || tableName.getSchemaName().equals(schemaNameOrNull)) { + builder.add(tableName); + } + } + + return builder.build(); + } + + @SuppressWarnings("ValueOfIncrementOrDecrementUsed") + @Override + public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) + { + KafkaTableHandle kafkaTableHandle = convertTableHandle(tableHandle); + + KafkaTopicDescription kafkaTopicDescription = tableDescriptions.get(kafkaTableHandle.toSchemaTableName()); + if (kafkaTopicDescription == null) { + throw new TableNotFoundException(kafkaTableHandle.toSchemaTableName()); + } + + ImmutableMap.Builder columnHandles = ImmutableMap.builder(); + + int index = 0; + KafkaTopicFieldGroup key = kafkaTopicDescription.getKey(); + if (key != null) { + List fields = key.getFields(); + if (fields != null) { + for (KafkaTopicFieldDescription kafkaTopicFieldDescription : fields) { + columnHandles.put(kafkaTopicFieldDescription.getName(), kafkaTopicFieldDescription.getColumnHandle(connectorId, true, index++)); + } + } + } + + KafkaTopicFieldGroup message = kafkaTopicDescription.getMessage(); + if (message != null) { + List fields = message.getFields(); + if (fields != null) { + for (KafkaTopicFieldDescription kafkaTopicFieldDescription : fields) { + columnHandles.put(kafkaTopicFieldDescription.getName(), kafkaTopicFieldDescription.getColumnHandle(connectorId, false, index++)); + } + } + } + + for (KafkaInternalFieldDescription kafkaInternalFieldDescription : internalFieldDescriptions) { + columnHandles.put(kafkaInternalFieldDescription.getName(), kafkaInternalFieldDescription.getColumnHandle(connectorId, index++, hideInternalColumns)); + } + + return columnHandles.build(); + } + + @Override + public Map> listTableColumns(ConnectorSession session, SchemaTablePrefix prefix) + { + requireNonNull(prefix, "prefix is null"); + + ImmutableMap.Builder> columns = ImmutableMap.builder(); + + List tableNames = prefix.getSchemaName() == null ? listTables(session, null) : ImmutableList.of(new SchemaTableName(prefix.getSchemaName(), prefix.getTableName())); + + for (SchemaTableName tableName : tableNames) { + ConnectorTableMetadata tableMetadata = getTableMetadata(tableName); + // table can disappear during listing operation + if (tableMetadata != null) { + columns.put(tableName, tableMetadata.getColumns()); + } + } + return columns.build(); + } + + @Override + public ColumnMetadata getColumnMetadata(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnHandle columnHandle) + { + convertTableHandle(tableHandle); + return convertColumnHandle(columnHandle).getColumnMetadata(); + } + + @Override + public List getTableLayouts(ConnectorSession session, ConnectorTableHandle table, Constraint constraint, Optional> desiredColumns) + { + KafkaTableHandle handle = convertTableHandle(table); + ConnectorTableLayout layout = new ConnectorTableLayout(new KafkaTableLayoutHandle(handle)); + return ImmutableList.of(new ConnectorTableLayoutResult(layout, constraint.getSummary())); + } + + @Override + public ConnectorTableLayout getTableLayout(ConnectorSession session, ConnectorTableLayoutHandle handle) + { + return new ConnectorTableLayout(handle); + } + + @SuppressWarnings("ValueOfIncrementOrDecrementUsed") + private ConnectorTableMetadata getTableMetadata(SchemaTableName schemaTableName) + { + KafkaTopicDescription table = tableDescriptions.get(schemaTableName); + if (table == null) { + throw new TableNotFoundException(schemaTableName); + } + + ImmutableList.Builder builder = ImmutableList.builder(); + + KafkaTopicFieldGroup key = table.getKey(); + if (key != null) { + List fields = key.getFields(); + if (fields != null) { + for (KafkaTopicFieldDescription fieldDescription : fields) { + builder.add(fieldDescription.getColumnMetadata()); + } + } + } + + KafkaTopicFieldGroup message = table.getMessage(); + if (message != null) { + List fields = message.getFields(); + if (fields != null) { + for (KafkaTopicFieldDescription fieldDescription : fields) { + builder.add(fieldDescription.getColumnMetadata()); + } + } + } + + for (KafkaInternalFieldDescription fieldDescription : internalFieldDescriptions) { + builder.add(fieldDescription.getColumnMetadata(hideInternalColumns)); + } + + return new ConnectorTableMetadata(schemaTableName, builder.build()); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaPlugin.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaPlugin.java new file mode 100644 index 000000000000..284424a9a7fa --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaPlugin.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.kafka; + +import com.facebook.presto.spi.Plugin; +import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.spi.connector.ConnectorFactory; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; + +import java.util.Map; +import java.util.Optional; +import java.util.function.Supplier; + +import static java.util.Objects.requireNonNull; + +/** + * Presto plugin to use Apache Kafka as a data source. + */ +public class KafkaPlugin + implements Plugin +{ + private Optional>> tableDescriptionSupplier = Optional.empty(); + + @VisibleForTesting + public synchronized void setTableDescriptionSupplier(Supplier> tableDescriptionSupplier) + { + this.tableDescriptionSupplier = Optional.of(requireNonNull(tableDescriptionSupplier, "tableDescriptionSupplier is null")); + } + + @Override + public synchronized Iterable getConnectorFactories() + { + return ImmutableList.of(new KafkaConnectorFactory(tableDescriptionSupplier)); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java new file mode 100644 index 000000000000..cb24bd1e031e --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java @@ -0,0 +1,328 @@ +/* + * 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.kafka; + +import com.facebook.presto.decoder.DecoderColumnHandle; +import com.facebook.presto.decoder.FieldDecoder; +import com.facebook.presto.decoder.FieldValueProvider; +import com.facebook.presto.decoder.RowDecoder; +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.RecordCursor; +import com.facebook.presto.spi.RecordSet; +import com.facebook.presto.spi.type.Type; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import io.airlift.log.Logger; +import io.airlift.slice.Slice; +import io.airlift.slice.Slices; +import kafka.api.FetchRequest; +import kafka.api.FetchRequestBuilder; +import kafka.javaapi.FetchResponse; +import kafka.javaapi.consumer.SimpleConsumer; +import kafka.message.MessageAndOffset; + +import java.nio.ByteBuffer; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; + +import static com.facebook.presto.kafka.KafkaErrorCode.KAFKA_SPLIT_ERROR; +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; + +/** + * Kafka specific record set. Returns a cursor for a topic which iterates over a Kafka partition segment. + */ +public class KafkaRecordSet + implements RecordSet +{ + private static final Logger log = Logger.get(KafkaRecordSet.class); + + private static final int KAFKA_READ_BUFFER_SIZE = 100_000; + private static final byte [] EMPTY_BYTE_ARRAY = new byte [0]; + + private final KafkaSplit split; + private final KafkaSimpleConsumerManager consumerManager; + + private final RowDecoder keyDecoder; + private final RowDecoder messageDecoder; + private final Map> keyFieldDecoders; + private final Map> messageFieldDecoders; + + private final List columnHandles; + private final List columnTypes; + + private final Set globalInternalFieldValueProviders; + + KafkaRecordSet(KafkaSplit split, + KafkaSimpleConsumerManager consumerManager, + List columnHandles, + RowDecoder keyDecoder, + RowDecoder messageDecoder, + Map> keyFieldDecoders, + Map> messageFieldDecoders) + { + this.split = requireNonNull(split, "split is null"); + + this.globalInternalFieldValueProviders = ImmutableSet.of( + KafkaInternalFieldDescription.PARTITION_ID_FIELD.forLongValue(split.getPartitionId()), + KafkaInternalFieldDescription.SEGMENT_START_FIELD.forLongValue(split.getStart()), + KafkaInternalFieldDescription.SEGMENT_END_FIELD.forLongValue(split.getEnd())); + + this.consumerManager = requireNonNull(consumerManager, "consumerManager is null"); + + this.keyDecoder = requireNonNull(keyDecoder, "rowDecoder is null"); + this.messageDecoder = requireNonNull(messageDecoder, "rowDecoder is null"); + this.keyFieldDecoders = requireNonNull(keyFieldDecoders, "keyFieldDecoders is null"); + this.messageFieldDecoders = requireNonNull(messageFieldDecoders, "messageFieldDecoders is null"); + + this.columnHandles = requireNonNull(columnHandles, "columnHandles is null"); + + ImmutableList.Builder typeBuilder = ImmutableList.builder(); + + for (DecoderColumnHandle handle : columnHandles) { + typeBuilder.add(handle.getType()); + } + + this.columnTypes = typeBuilder.build(); + } + + @Override + public List getColumnTypes() + { + return columnTypes; + } + + @Override + public RecordCursor cursor() + { + return new KafkaRecordCursor(); + } + + public class KafkaRecordCursor + implements RecordCursor + { + private long totalBytes; + private long totalMessages; + private long cursorOffset = split.getStart(); + private Iterator messageAndOffsetIterator; + private final AtomicBoolean reported = new AtomicBoolean(); + + private FieldValueProvider[] fieldValueProviders; + + KafkaRecordCursor() + { + } + + @Override + public long getTotalBytes() + { + return totalBytes; + } + + @Override + public long getCompletedBytes() + { + return totalBytes; + } + + @Override + public long getReadTimeNanos() + { + return 0; + } + + @Override + public Type getType(int field) + { + checkArgument(field < columnHandles.size(), "Invalid field index"); + return columnHandles.get(field).getType(); + } + + @Override + public boolean advanceNextPosition() + { + while (true) { + if (cursorOffset >= split.getEnd()) { + return endOfData(); // Split end is exclusive. + } + // Create a fetch request + openFetchRequest(); + + while (messageAndOffsetIterator.hasNext()) { + MessageAndOffset currentMessageAndOffset = messageAndOffsetIterator.next(); + long messageOffset = currentMessageAndOffset.offset(); + + if (messageOffset >= split.getEnd()) { + return endOfData(); // Past our split end. Bail. + } + + if (messageOffset >= cursorOffset) { + return nextRow(currentMessageAndOffset); + } + } + messageAndOffsetIterator = null; + } + } + + private boolean endOfData() + { + if (!reported.getAndSet(true)) { + log.debug("Found a total of %d messages with %d bytes (%d messages expected). Last Offset: %d (%d, %d)", + totalMessages, totalBytes, split.getEnd() - split.getStart(), + cursorOffset, split.getStart(), split.getEnd()); + } + return false; + } + + private boolean nextRow(MessageAndOffset messageAndOffset) + { + cursorOffset = messageAndOffset.offset() + 1; // Cursor now points to the next message. + totalBytes += messageAndOffset.message().payloadSize(); + totalMessages++; + + byte[] keyData = EMPTY_BYTE_ARRAY; + byte[] messageData = EMPTY_BYTE_ARRAY; + ByteBuffer key = messageAndOffset.message().key(); + if (key != null) { + keyData = new byte[key.remaining()]; + key.get(keyData); + } + + ByteBuffer message = messageAndOffset.message().payload(); + if (message != null) { + messageData = new byte[message.remaining()]; + message.get(messageData); + } + + Set fieldValueProviders = new HashSet<>(); + + fieldValueProviders.addAll(globalInternalFieldValueProviders); + fieldValueProviders.add(KafkaInternalFieldDescription.SEGMENT_COUNT_FIELD.forLongValue(totalMessages)); + fieldValueProviders.add(KafkaInternalFieldDescription.PARTITION_OFFSET_FIELD.forLongValue(messageAndOffset.offset())); + fieldValueProviders.add(KafkaInternalFieldDescription.MESSAGE_FIELD.forByteValue(messageData)); + fieldValueProviders.add(KafkaInternalFieldDescription.MESSAGE_LENGTH_FIELD.forLongValue(messageData.length)); + fieldValueProviders.add(KafkaInternalFieldDescription.KEY_FIELD.forByteValue(keyData)); + fieldValueProviders.add(KafkaInternalFieldDescription.KEY_LENGTH_FIELD.forLongValue(keyData.length)); + fieldValueProviders.add(KafkaInternalFieldDescription.KEY_CORRUPT_FIELD.forBooleanValue(keyDecoder.decodeRow(keyData, null, fieldValueProviders, columnHandles, keyFieldDecoders))); + fieldValueProviders.add(KafkaInternalFieldDescription.MESSAGE_CORRUPT_FIELD.forBooleanValue(messageDecoder.decodeRow(messageData, null, fieldValueProviders, columnHandles, messageFieldDecoders))); + + this.fieldValueProviders = new FieldValueProvider[columnHandles.size()]; + + // If a value provider for a requested internal column is present, assign the + // value to the internal cache. It is possible that an internal column is present + // where no value provider exists (e.g. the '_corrupt' column with the DummyRowDecoder). + // In that case, the cache is null (and the column is reported as null). + for (int i = 0; i < columnHandles.size(); i++) { + for (FieldValueProvider fieldValueProvider : fieldValueProviders) { + if (fieldValueProvider.accept(columnHandles.get(i))) { + this.fieldValueProviders[i] = fieldValueProvider; + break; // for(InternalColumnProvider... + } + } + } + + return true; // Advanced successfully. + } + + @SuppressWarnings("SimplifiableConditionalExpression") + @Override + public boolean getBoolean(int field) + { + checkArgument(field < columnHandles.size(), "Invalid field index"); + + checkFieldType(field, boolean.class); + return isNull(field) ? false : fieldValueProviders[field].getBoolean(); + } + + @Override + public long getLong(int field) + { + checkArgument(field < columnHandles.size(), "Invalid field index"); + + checkFieldType(field, long.class); + return isNull(field) ? 0L : fieldValueProviders[field].getLong(); + } + + @Override + public double getDouble(int field) + { + checkArgument(field < columnHandles.size(), "Invalid field index"); + + checkFieldType(field, double.class); + return isNull(field) ? 0.0d : fieldValueProviders[field].getDouble(); + } + + @Override + public Slice getSlice(int field) + { + checkArgument(field < columnHandles.size(), "Invalid field index"); + + checkFieldType(field, Slice.class); + return isNull(field) ? Slices.EMPTY_SLICE : fieldValueProviders[field].getSlice(); + } + + @Override + public Object getObject(int field) + { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isNull(int field) + { + checkArgument(field < columnHandles.size(), "Invalid field index"); + + return fieldValueProviders[field] == null || fieldValueProviders[field].isNull(); + } + + private void checkFieldType(int field, Class expected) + { + Class actual = getType(field).getJavaType(); + checkArgument(actual == expected, "Expected field %s to be type %s but is %s", field, expected, actual); + } + + @Override + public void close() + { + } + + private void openFetchRequest() + { + if (messageAndOffsetIterator == null) { + log.debug("Fetching %d bytes from offset %d (%d - %d). %d messages read so far", KAFKA_READ_BUFFER_SIZE, cursorOffset, split.getStart(), split.getEnd(), totalMessages); + FetchRequest req = new FetchRequestBuilder() + .clientId("presto-worker-" + Thread.currentThread().getName()) + .addFetch(split.getTopicName(), split.getPartitionId(), cursorOffset, KAFKA_READ_BUFFER_SIZE) + .build(); + + // TODO - this should look at the actual node this is running on and prefer + // that copy if running locally. - look into NodeInfo + SimpleConsumer consumer = consumerManager.getConsumer(split.getLeader()); + + FetchResponse fetchResponse = consumer.fetch(req); + if (fetchResponse.hasError()) { + short errorCode = fetchResponse.errorCode(split.getTopicName(), split.getPartitionId()); + log.warn("Fetch response has error: %d", errorCode); + throw new PrestoException(KAFKA_SPLIT_ERROR, "could not fetch data from Kafka, error code is '" + errorCode + "'"); + } + + messageAndOffsetIterator = fetchResponse.messageSet(split.getTopicName(), split.getPartitionId()).iterator(); + } + } + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSetProvider.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSetProvider.java new file mode 100644 index 000000000000..451cab049d46 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSetProvider.java @@ -0,0 +1,95 @@ +/* + * 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.kafka; + +import com.facebook.presto.decoder.DecoderColumnHandle; +import com.facebook.presto.decoder.DecoderRegistry; +import com.facebook.presto.decoder.FieldDecoder; +import com.facebook.presto.decoder.RowDecoder; +import com.facebook.presto.spi.ColumnHandle; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.ConnectorSplit; +import com.facebook.presto.spi.RecordSet; +import com.facebook.presto.spi.connector.ConnectorRecordSetProvider; +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; + +import javax.inject.Inject; + +import java.util.List; + +import static com.facebook.presto.kafka.KafkaHandleResolver.convertColumnHandle; +import static com.facebook.presto.kafka.KafkaHandleResolver.convertSplit; +import static java.util.Objects.requireNonNull; + +/** + * Factory for Kafka specific {@link RecordSet} instances. + */ +public class KafkaRecordSetProvider + implements ConnectorRecordSetProvider +{ + private final KafkaSimpleConsumerManager consumerManager; + private final DecoderRegistry registry; + + @Inject + public KafkaRecordSetProvider(DecoderRegistry registry, KafkaSimpleConsumerManager consumerManager) + { + this.registry = requireNonNull(registry, "registry is null"); + this.consumerManager = requireNonNull(consumerManager, "consumerManager is null"); + } + + @Override + public RecordSet getRecordSet(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorSplit split, List columns) + { + KafkaSplit kafkaSplit = convertSplit(split); + + ImmutableList.Builder handleBuilder = ImmutableList.builder(); + ImmutableMap.Builder> keyFieldDecoderBuilder = ImmutableMap.builder(); + ImmutableMap.Builder> messageFieldDecoderBuilder = ImmutableMap.builder(); + + RowDecoder keyDecoder = registry.getRowDecoder(kafkaSplit.getKeyDataFormat()); + RowDecoder messageDecoder = registry.getRowDecoder(kafkaSplit.getMessageDataFormat()); + + for (ColumnHandle handle : columns) { + KafkaColumnHandle columnHandle = convertColumnHandle(handle); + handleBuilder.add(columnHandle); + + if (!columnHandle.isInternal()) { + if (columnHandle.isKeyDecoder()) { + FieldDecoder fieldDecoder = registry.getFieldDecoder( + kafkaSplit.getKeyDataFormat(), + columnHandle.getType().getJavaType(), + columnHandle.getDataFormat()); + + keyFieldDecoderBuilder.put(columnHandle, fieldDecoder); + } + else { + FieldDecoder fieldDecoder = registry.getFieldDecoder( + kafkaSplit.getMessageDataFormat(), + columnHandle.getType().getJavaType(), + columnHandle.getDataFormat()); + + messageFieldDecoderBuilder.put(columnHandle, fieldDecoder); + } + } + } + + ImmutableList handles = handleBuilder.build(); + ImmutableMap> keyFieldDecoders = keyFieldDecoderBuilder.build(); + ImmutableMap> messageFieldDecoders = messageFieldDecoderBuilder.build(); + + return new KafkaRecordSet(kafkaSplit, consumerManager, handles, keyDecoder, messageDecoder, keyFieldDecoders, messageFieldDecoders); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSimpleConsumerManager.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSimpleConsumerManager.java new file mode 100644 index 000000000000..5c4ebb023d47 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSimpleConsumerManager.java @@ -0,0 +1,105 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.kafka; + +import com.facebook.presto.spi.HostAddress; +import com.facebook.presto.spi.NodeManager; +import com.google.common.base.Throwables; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; +import io.airlift.log.Logger; +import kafka.javaapi.consumer.SimpleConsumer; + +import javax.annotation.PreDestroy; +import javax.inject.Inject; + +import java.util.Map; +import java.util.concurrent.ExecutionException; + +import static java.lang.Math.toIntExact; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +/** + * Manages connections to the Kafka nodes. A worker may connect to multiple Kafka nodes depending on the segments and partitions + * it needs to process. According to the Kafka source code, a Kafka {@link kafka.javaapi.consumer.SimpleConsumer} is thread-safe. + */ +public class KafkaSimpleConsumerManager +{ + private static final Logger log = Logger.get(KafkaSimpleConsumerManager.class); + + private final LoadingCache consumerCache; + + private final String connectorId; + private final NodeManager nodeManager; + private final int connectTimeoutMillis; + private final int bufferSizeBytes; + + @Inject + public KafkaSimpleConsumerManager( + KafkaConnectorId connectorId, + KafkaConnectorConfig kafkaConnectorConfig, + NodeManager nodeManager) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); + this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); + + requireNonNull(kafkaConnectorConfig, "kafkaConfig is null"); + this.connectTimeoutMillis = toIntExact(kafkaConnectorConfig.getKafkaConnectTimeout().toMillis()); + this.bufferSizeBytes = toIntExact(kafkaConnectorConfig.getKafkaBufferSize().toBytes()); + + this.consumerCache = CacheBuilder.newBuilder().build(new SimpleConsumerCacheLoader()); + } + + @PreDestroy + public void tearDown() + { + for (Map.Entry entry : consumerCache.asMap().entrySet()) { + try { + entry.getValue().close(); + } + catch (Exception e) { + log.warn(e, "While closing consumer %s:", entry.getKey()); + } + } + } + + public SimpleConsumer getConsumer(HostAddress host) + { + requireNonNull(host, "host is null"); + try { + return consumerCache.get(host); + } + catch (ExecutionException e) { + throw Throwables.propagate(e.getCause()); + } + } + + private class SimpleConsumerCacheLoader + extends CacheLoader + { + @Override + public SimpleConsumer load(HostAddress host) + throws Exception + { + log.info("Creating new Consumer for %s", host); + return new SimpleConsumer(host.getHostText(), + host.getPort(), + connectTimeoutMillis, + bufferSizeBytes, + format("presto-kafka-%s-%s", connectorId, nodeManager.getCurrentNode().getNodeIdentifier())); + } + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplit.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplit.java new file mode 100644 index 000000000000..ac61361a4287 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplit.java @@ -0,0 +1,147 @@ +/* + * 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.kafka; + +import com.facebook.presto.spi.ConnectorSplit; +import com.facebook.presto.spi.HostAddress; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; + +import java.util.List; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +/** + * Represents a kafka specific {@link ConnectorSplit}. Each split is mapped to a segment file on disk (based off the segment offset start() and end() values) so that + * a partition can be processed by reading segment files from partition leader. Otherwise, a Kafka topic could only be processed along partition boundaries. + *

+ * When planning to process a Kafka topic with Presto, using smaller than the recommended segment size (default is 1G) allows Presto to optimize early and process a topic + * with more workers in parallel. + */ +public class KafkaSplit + implements ConnectorSplit +{ + private final String connectorId; + private final String topicName; + private final String keyDataFormat; + private final String messageDataFormat; + private final int partitionId; + private final long start; + private final long end; + private final HostAddress leader; + + @JsonCreator + public KafkaSplit( + @JsonProperty("connectorId") String connectorId, + @JsonProperty("topicName") String topicName, + @JsonProperty("keyDataFormat") String keyDataFormat, + @JsonProperty("messageDataFormat") String messageDataFormat, + @JsonProperty("partitionId") int partitionId, + @JsonProperty("start") long start, + @JsonProperty("end") long end, + @JsonProperty("leader") HostAddress leader) + { + this.connectorId = requireNonNull(connectorId, "connector id is null"); + this.topicName = requireNonNull(topicName, "topicName is null"); + this.keyDataFormat = requireNonNull(keyDataFormat, "dataFormat is null"); + this.messageDataFormat = requireNonNull(messageDataFormat, "messageDataFormat is null"); + this.partitionId = partitionId; + this.start = start; + this.end = end; + this.leader = requireNonNull(leader, "leader address is null"); + } + + @JsonProperty + public String getConnectorId() + { + return connectorId; + } + + @JsonProperty + public long getStart() + { + return start; + } + + @JsonProperty + public long getEnd() + { + return end; + } + + @JsonProperty + public String getTopicName() + { + return topicName; + } + + @JsonProperty + public String getKeyDataFormat() + { + return keyDataFormat; + } + + @JsonProperty + public String getMessageDataFormat() + { + return messageDataFormat; + } + + @JsonProperty + public int getPartitionId() + { + return partitionId; + } + + @JsonProperty + public HostAddress getLeader() + { + return leader; + } + + @Override + public boolean isRemotelyAccessible() + { + return true; + } + + @Override + public List getAddresses() + { + return ImmutableList.of(leader); + } + + @Override + public Object getInfo() + { + return this; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("connectorId", connectorId) + .add("topicName", topicName) + .add("keyDataFormat", keyDataFormat) + .add("messageDataFormat", messageDataFormat) + .add("partitionId", partitionId) + .add("start", start) + .add("end", end) + .add("leader", leader) + .toString(); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java new file mode 100644 index 000000000000..f7b436a97b14 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java @@ -0,0 +1,149 @@ +/* + * 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.kafka; + +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.ConnectorSplit; +import com.facebook.presto.spi.ConnectorSplitSource; +import com.facebook.presto.spi.ConnectorTableLayoutHandle; +import com.facebook.presto.spi.FixedSplitSource; +import com.facebook.presto.spi.HostAddress; +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.connector.ConnectorSplitManager; +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import io.airlift.log.Logger; +import kafka.api.PartitionOffsetRequestInfo; +import kafka.cluster.Broker; +import kafka.common.TopicAndPartition; +import kafka.javaapi.OffsetRequest; +import kafka.javaapi.OffsetResponse; +import kafka.javaapi.PartitionMetadata; +import kafka.javaapi.TopicMetadata; +import kafka.javaapi.TopicMetadataRequest; +import kafka.javaapi.TopicMetadataResponse; +import kafka.javaapi.consumer.SimpleConsumer; + +import javax.inject.Inject; + +import java.util.List; +import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; + +import static com.facebook.presto.kafka.KafkaErrorCode.KAFKA_SPLIT_ERROR; +import static com.facebook.presto.kafka.KafkaHandleResolver.convertLayout; +import static java.util.Objects.requireNonNull; + +/** + * Kafka specific implementation of {@link ConnectorSplitManager}. + */ +public class KafkaSplitManager + implements ConnectorSplitManager +{ + private static final Logger log = Logger.get(KafkaSplitManager.class); + + private final String connectorId; + private final KafkaSimpleConsumerManager consumerManager; + private final Set nodes; + + @Inject + public KafkaSplitManager( + KafkaConnectorId connectorId, + KafkaConnectorConfig kafkaConnectorConfig, + KafkaSimpleConsumerManager consumerManager) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); + this.consumerManager = requireNonNull(consumerManager, "consumerManager is null"); + + requireNonNull(kafkaConnectorConfig, "kafkaConfig is null"); + this.nodes = ImmutableSet.copyOf(kafkaConnectorConfig.getNodes()); + } + + @Override + public ConnectorSplitSource getSplits(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableLayoutHandle layout) + { + KafkaTableHandle kafkaTableHandle = convertLayout(layout).getTable(); + + SimpleConsumer simpleConsumer = consumerManager.getConsumer(selectRandom(nodes)); + + TopicMetadataRequest topicMetadataRequest = new TopicMetadataRequest(ImmutableList.of(kafkaTableHandle.getTopicName())); + TopicMetadataResponse topicMetadataResponse = simpleConsumer.send(topicMetadataRequest); + + ImmutableList.Builder splits = ImmutableList.builder(); + + for (TopicMetadata metadata : topicMetadataResponse.topicsMetadata()) { + for (PartitionMetadata part : metadata.partitionsMetadata()) { + log.debug("Adding Partition %s/%s", metadata.topic(), part.partitionId()); + + Broker leader = part.leader(); + if (leader == null) { // Leader election going on... + log.warn("No leader for partition %s/%s found!", metadata.topic(), part.partitionId()); + continue; + } + + HostAddress partitionLeader = HostAddress.fromParts(leader.host(), leader.port()); + + SimpleConsumer leaderConsumer = consumerManager.getConsumer(partitionLeader); + // Kafka contains a reverse list of "end - start" pairs for the splits + + long[] offsets = findAllOffsets(leaderConsumer, metadata.topic(), part.partitionId()); + + for (int i = offsets.length - 1; i > 0; i--) { + KafkaSplit split = new KafkaSplit( + connectorId, + metadata.topic(), + kafkaTableHandle.getKeyDataFormat(), + kafkaTableHandle.getMessageDataFormat(), + part.partitionId(), + offsets[i], + offsets[i - 1], + partitionLeader); + splits.add(split); + } + } + } + + return new FixedSplitSource(splits.build()); + } + + private static long[] findAllOffsets(SimpleConsumer consumer, String topicName, int partitionId) + { + TopicAndPartition topicAndPartition = new TopicAndPartition(topicName, partitionId); + + // The API implies that this will always return all of the offsets. So it seems a partition can not have + // more than Integer.MAX_VALUE-1 segments. + // + // This also assumes that the lowest value returned will be the first segment available. So if segments have been dropped off, this value + // should not be 0. + PartitionOffsetRequestInfo partitionOffsetRequestInfo = new PartitionOffsetRequestInfo(kafka.api.OffsetRequest.LatestTime(), Integer.MAX_VALUE); + OffsetRequest offsetRequest = new OffsetRequest(ImmutableMap.of(topicAndPartition, partitionOffsetRequestInfo), kafka.api.OffsetRequest.CurrentVersion(), consumer.clientId()); + OffsetResponse offsetResponse = consumer.getOffsetsBefore(offsetRequest); + + if (offsetResponse.hasError()) { + short errorCode = offsetResponse.errorCode(topicName, partitionId); + log.warn("Offset response has error: %d", errorCode); + throw new PrestoException(KAFKA_SPLIT_ERROR, "could not fetch data from Kafka, error code is '" + errorCode + "'"); + } + + return offsetResponse.offsets(topicName, partitionId); + } + + private static T selectRandom(Iterable iterable) + { + List list = ImmutableList.copyOf(iterable); + return list.get(ThreadLocalRandom.current().nextInt(list.size())); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableDescriptionSupplier.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableDescriptionSupplier.java new file mode 100644 index 000000000000..2fcdbe539ef4 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableDescriptionSupplier.java @@ -0,0 +1,138 @@ +/* + * 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.kafka; + +import com.facebook.presto.decoder.dummy.DummyRowDecoder; +import com.facebook.presto.spi.SchemaTableName; +import com.google.common.base.Splitter; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import io.airlift.json.JsonCodec; +import io.airlift.log.Logger; + +import javax.inject.Inject; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Supplier; + +import static com.google.common.base.MoreObjects.firstNonNull; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Strings.isNullOrEmpty; +import static java.nio.file.Files.readAllBytes; +import static java.util.Arrays.asList; +import static java.util.Objects.requireNonNull; + +public class KafkaTableDescriptionSupplier + implements Supplier> +{ + private static final Logger log = Logger.get(KafkaTableDescriptionSupplier.class); + + private final JsonCodec topicDescriptionCodec; + private final File tableDescriptionDir; + private final String defaultSchema; + private final Set tableNames; + + @Inject + KafkaTableDescriptionSupplier(KafkaConnectorConfig kafkaConnectorConfig, + JsonCodec topicDescriptionCodec) + { + this.topicDescriptionCodec = requireNonNull(topicDescriptionCodec, "topicDescriptionCodec is null"); + + requireNonNull(kafkaConnectorConfig, "kafkaConfig is null"); + this.tableDescriptionDir = kafkaConnectorConfig.getTableDescriptionDir(); + this.defaultSchema = kafkaConnectorConfig.getDefaultSchema(); + this.tableNames = ImmutableSet.copyOf(kafkaConnectorConfig.getTableNames()); + } + + @Override + public Map get() + { + ImmutableMap.Builder builder = ImmutableMap.builder(); + + log.debug("Loading kafka table definitions from %s", tableDescriptionDir.getAbsolutePath()); + + try { + for (File file : listFiles(tableDescriptionDir)) { + if (file.isFile() && file.getName().endsWith(".json")) { + KafkaTopicDescription table = topicDescriptionCodec.fromJson(readAllBytes(file.toPath())); + String schemaName = firstNonNull(table.getSchemaName(), defaultSchema); + log.debug("Kafka table %s.%s: %s", schemaName, table.getTableName(), table); + builder.put(new SchemaTableName(schemaName, table.getTableName()), table); + } + } + + Map tableDefinitions = builder.build(); + + log.debug("Loaded Table definitions: %s", tableDefinitions.keySet()); + + builder = ImmutableMap.builder(); + for (String definedTable : tableNames) { + SchemaTableName tableName; + try { + tableName = parseTableName(definedTable); + } + catch (IllegalArgumentException iae) { + tableName = new SchemaTableName(defaultSchema, definedTable); + } + + if (tableDefinitions.containsKey(tableName)) { + KafkaTopicDescription kafkaTable = tableDefinitions.get(tableName); + log.debug("Found Table definition for %s: %s", tableName, kafkaTable); + builder.put(tableName, kafkaTable); + } + else { + // A dummy table definition only supports the internal columns. + log.debug("Created dummy Table definition for %s", tableName); + builder.put(tableName, new KafkaTopicDescription(tableName.getTableName(), + tableName.getSchemaName(), + definedTable, + new KafkaTopicFieldGroup(DummyRowDecoder.NAME, ImmutableList.of()), + new KafkaTopicFieldGroup(DummyRowDecoder.NAME, ImmutableList.of()))); + } + } + + return builder.build(); + } + catch (IOException e) { + log.warn(e, "Error: "); + throw Throwables.propagate(e); + } + } + + private static List listFiles(File dir) + { + if ((dir != null) && dir.isDirectory()) { + File[] files = dir.listFiles(); + if (files != null) { + log.debug("Considering files: %s", asList(files)); + return ImmutableList.copyOf(files); + } + } + return ImmutableList.of(); + } + + private static SchemaTableName parseTableName(String schemaTableName) + { + checkArgument(!isNullOrEmpty(schemaTableName), "schemaTableName is null or is empty"); + List parts = Splitter.on('.').splitToList(schemaTableName); + checkArgument(parts.size() == 2, "Invalid schemaTableName: %s", schemaTableName); + return new SchemaTableName(parts.get(0), parts.get(1)); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableHandle.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableHandle.java new file mode 100644 index 000000000000..8475b2c68c98 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableHandle.java @@ -0,0 +1,151 @@ +/* + * 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.kafka; + +import com.facebook.presto.spi.ConnectorTableHandle; +import com.facebook.presto.spi.SchemaTableName; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +/** + * Kafka specific {@link ConnectorTableHandle}. + */ +public final class KafkaTableHandle + implements ConnectorTableHandle +{ + /** + * connector id + */ + private final String connectorId; + + /** + * The schema name for this table. Is set through configuration and read + * using {@link KafkaConnectorConfig#getDefaultSchema()}. Usually 'default'. + */ + private final String schemaName; + + /** + * The table name used by presto. + */ + private final String tableName; + + /** + * The topic name that is read from Kafka. + */ + private final String topicName; + + private final String keyDataFormat; + private final String messageDataFormat; + + @JsonCreator + public KafkaTableHandle( + @JsonProperty("connectorId") String connectorId, + @JsonProperty("schemaName") String schemaName, + @JsonProperty("tableName") String tableName, + @JsonProperty("topicName") String topicName, + @JsonProperty("keyDataFormat") String keyDataFormat, + @JsonProperty("messageDataFormat") String messageDataFormat) + { + this.connectorId = requireNonNull(connectorId, "connectorId is null"); + this.schemaName = requireNonNull(schemaName, "schemaName is null"); + this.tableName = requireNonNull(tableName, "tableName is null"); + this.topicName = requireNonNull(topicName, "topicName is null"); + this.keyDataFormat = requireNonNull(keyDataFormat, "keyDataFormat is null"); + this.messageDataFormat = requireNonNull(messageDataFormat, "messageDataFormat is null"); + } + + @JsonProperty + public String getConnectorId() + { + return connectorId; + } + + @JsonProperty + public String getSchemaName() + { + return schemaName; + } + + @JsonProperty + public String getTableName() + { + return tableName; + } + + @JsonProperty + public String getTopicName() + { + return topicName; + } + + @JsonProperty + public String getKeyDataFormat() + { + return keyDataFormat; + } + + @JsonProperty + public String getMessageDataFormat() + { + return messageDataFormat; + } + + public SchemaTableName toSchemaTableName() + { + return new SchemaTableName(schemaName, tableName); + } + + @Override + public int hashCode() + { + return Objects.hash(connectorId, schemaName, tableName, topicName, keyDataFormat, messageDataFormat); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + KafkaTableHandle other = (KafkaTableHandle) obj; + return Objects.equals(this.connectorId, other.connectorId) + && Objects.equals(this.schemaName, other.schemaName) + && Objects.equals(this.tableName, other.tableName) + && Objects.equals(this.topicName, other.topicName) + && Objects.equals(this.keyDataFormat, other.keyDataFormat) + && Objects.equals(this.messageDataFormat, other.messageDataFormat); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("connectorId", connectorId) + .add("schemaName", schemaName) + .add("tableName", tableName) + .add("topicName", topicName) + .add("keyDataFormat", keyDataFormat) + .add("messageDataFormat", messageDataFormat) + .toString(); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableLayoutHandle.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableLayoutHandle.java new file mode 100644 index 000000000000..de0c37816b23 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableLayoutHandle.java @@ -0,0 +1,44 @@ +/* + * 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.kafka; + +import com.facebook.presto.spi.ConnectorTableLayoutHandle; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import static java.util.Objects.requireNonNull; + +public class KafkaTableLayoutHandle + implements ConnectorTableLayoutHandle +{ + private final KafkaTableHandle table; + + @JsonCreator + public KafkaTableLayoutHandle(@JsonProperty("table") KafkaTableHandle table) + { + this.table = requireNonNull(table, "table is null"); + } + + @JsonProperty + public KafkaTableHandle getTable() + { + return table; + } + + @Override + public String toString() + { + return table.toString(); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicDescription.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicDescription.java new file mode 100644 index 000000000000..443bb6828940 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicDescription.java @@ -0,0 +1,92 @@ +/* + * 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.kafka; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Strings.isNullOrEmpty; +import static java.util.Objects.requireNonNull; + +/** + * Json description to parse a row on a Kafka topic. A row contains a message and an optional key. See the documentation for the exact JSON syntax. + */ +public class KafkaTopicDescription +{ + private final String tableName; + private final String topicName; + private final String schemaName; + private final KafkaTopicFieldGroup key; + private final KafkaTopicFieldGroup message; + + @JsonCreator + public KafkaTopicDescription( + @JsonProperty("tableName") String tableName, + @JsonProperty("schemaName") String schemaName, + @JsonProperty("topicName") String topicName, + @JsonProperty("key") KafkaTopicFieldGroup key, + @JsonProperty("message") KafkaTopicFieldGroup message) + { + checkArgument(!isNullOrEmpty(tableName), "tableName is null or is empty"); + this.tableName = tableName; + this.topicName = requireNonNull(topicName, "topicName is null"); + this.schemaName = schemaName; + this.key = key; + this.message = message; + } + + @JsonProperty + public String getTableName() + { + return tableName; + } + + @JsonProperty + public String getTopicName() + { + return topicName; + } + + @JsonProperty + public String getSchemaName() + { + return schemaName; + } + + @JsonProperty + public KafkaTopicFieldGroup getKey() + { + return key; + } + + @JsonProperty + public KafkaTopicFieldGroup getMessage() + { + return message; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("tableName", tableName) + .add("topicName", topicName) + .add("schemaName", schemaName) + .add("key", key) + .add("message", message) + .toString(); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicFieldDescription.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicFieldDescription.java new file mode 100644 index 000000000000..38c9861ea5f5 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicFieldDescription.java @@ -0,0 +1,159 @@ +/* + * 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.kafka; + +import com.facebook.presto.spi.ColumnMetadata; +import com.facebook.presto.spi.type.Type; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +import static com.google.common.base.MoreObjects.toStringHelper; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Strings.isNullOrEmpty; +import static java.util.Objects.requireNonNull; + +/** + * Json description to parse a single field from a Kafka topic message. See {@link com.facebook.presto.kafka.KafkaTopicDescription} for more details. + */ +public final class KafkaTopicFieldDescription +{ + private final String name; + private final Type type; + private final String mapping; + private final String comment; + private final String dataFormat; + private final String formatHint; + private final boolean hidden; + + @JsonCreator + public KafkaTopicFieldDescription( + @JsonProperty("name") String name, + @JsonProperty("type") Type type, + @JsonProperty("mapping") String mapping, + @JsonProperty("comment") String comment, + @JsonProperty("dataFormat") String dataFormat, + @JsonProperty("formatHint") String formatHint, + @JsonProperty("hidden") boolean hidden) + { + checkArgument(!isNullOrEmpty(name), "name is null or is empty"); + this.name = name; + this.type = requireNonNull(type, "type is null"); + this.mapping = mapping; + this.comment = comment; + this.dataFormat = dataFormat; + this.formatHint = formatHint; + this.hidden = hidden; + } + + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public Type getType() + { + return type; + } + + @JsonProperty + public String getMapping() + { + return mapping; + } + + @JsonProperty + public String getComment() + { + return comment; + } + + @JsonProperty + public String getDataFormat() + { + return dataFormat; + } + + @JsonProperty + public String getFormatHint() + { + return formatHint; + } + + @JsonProperty + public boolean isHidden() + { + return hidden; + } + + KafkaColumnHandle getColumnHandle(String connectorId, boolean keyDecoder, int index) + { + return new KafkaColumnHandle(connectorId, + index, + getName(), + getType(), + getMapping(), + getDataFormat(), + getFormatHint(), + keyDecoder, + isHidden(), + false); + } + + ColumnMetadata getColumnMetadata() + { + return new ColumnMetadata(getName(), getType(), getComment(), isHidden()); + } + + @Override + public int hashCode() + { + return Objects.hash(name, type, mapping, dataFormat, formatHint, hidden); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + KafkaTopicFieldDescription other = (KafkaTopicFieldDescription) obj; + return Objects.equals(this.name, other.name) && + Objects.equals(this.type, other.type) && + Objects.equals(this.mapping, other.mapping) && + Objects.equals(this.dataFormat, other.dataFormat) && + Objects.equals(this.formatHint, other.formatHint) && + Objects.equals(this.hidden, other.hidden); + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("name", name) + .add("type", type) + .add("mapping", mapping) + .add("dataFormat", dataFormat) + .add("formatHint", formatHint) + .add("hidden", hidden) + .toString(); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicFieldGroup.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicFieldGroup.java new file mode 100644 index 000000000000..4b5384fe982e --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicFieldGroup.java @@ -0,0 +1,62 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.kafka; + +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.MoreObjects.toStringHelper; +import static java.util.Objects.requireNonNull; + +/** + * Groups the field descriptions for message or key. + */ +public class KafkaTopicFieldGroup +{ + private final String dataFormat; + private final List fields; + + @JsonCreator + public KafkaTopicFieldGroup( + @JsonProperty("dataFormat") String dataFormat, + @JsonProperty("fields") List fields) + { + this.dataFormat = requireNonNull(dataFormat, "dataFormat is null"); + this.fields = ImmutableList.copyOf(requireNonNull(fields, "fields is null")); + } + + @JsonProperty + public String getDataFormat() + { + return dataFormat; + } + + @JsonProperty + public List getFields() + { + return fields; + } + + @Override + public String toString() + { + return toStringHelper(this) + .add("dataFormat", dataFormat) + .add("fields", fields) + .toString(); + } +} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTransactionHandle.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTransactionHandle.java new file mode 100644 index 000000000000..f777be380ef2 --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTransactionHandle.java @@ -0,0 +1,22 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.kafka; + +import com.facebook.presto.spi.connector.ConnectorTransactionHandle; + +public enum KafkaTransactionHandle + implements ConnectorTransactionHandle +{ + INSTANCE +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/KafkaQueryRunner.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/KafkaQueryRunner.java new file mode 100644 index 000000000000..1f760f8a4c46 --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/KafkaQueryRunner.java @@ -0,0 +1,142 @@ +/* + * 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.kafka; + +import com.facebook.presto.Session; +import com.facebook.presto.kafka.util.CodecSupplier; +import com.facebook.presto.kafka.util.EmbeddedKafka; +import com.facebook.presto.kafka.util.TestUtils; +import com.facebook.presto.metadata.Metadata; +import com.facebook.presto.metadata.QualifiedObjectName; +import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.tests.DistributedQueryRunner; +import com.facebook.presto.tests.TestingPrestoClient; +import com.facebook.presto.tpch.TpchPlugin; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.airlift.json.JsonCodec; +import io.airlift.log.Logger; +import io.airlift.log.Logging; +import io.airlift.tpch.TpchTable; + +import java.util.Map; + +import static com.facebook.presto.kafka.util.TestUtils.installKafkaPlugin; +import static com.facebook.presto.kafka.util.TestUtils.loadTpchTopicDescription; +import static com.facebook.presto.testing.TestingSession.testSessionBuilder; +import static com.facebook.presto.tpch.TpchMetadata.TINY_SCHEMA_NAME; +import static io.airlift.testing.Closeables.closeAllSuppress; +import static io.airlift.units.Duration.nanosSince; +import static java.util.Locale.ENGLISH; +import static java.util.concurrent.TimeUnit.SECONDS; + +public final class KafkaQueryRunner +{ + private KafkaQueryRunner() + { + } + + private static final Logger log = Logger.get("TestQueries"); + private static final String TPCH_SCHEMA = "tpch"; + + public static DistributedQueryRunner createKafkaQueryRunner(EmbeddedKafka embeddedKafka, TpchTable... tables) + throws Exception + { + return createKafkaQueryRunner(embeddedKafka, ImmutableList.copyOf(tables)); + } + + public static DistributedQueryRunner createKafkaQueryRunner(EmbeddedKafka embeddedKafka, Iterable> tables) + throws Exception + { + DistributedQueryRunner queryRunner = null; + try { + queryRunner = new DistributedQueryRunner(createSession(), 2); + + queryRunner.installPlugin(new TpchPlugin()); + queryRunner.createCatalog("tpch", "tpch"); + + embeddedKafka.start(); + + for (TpchTable table : tables) { + embeddedKafka.createTopics(kafkaTopicName(table)); + } + + Map topicDescriptions = createTpchTopicDescriptions(queryRunner.getCoordinator().getMetadata(), tables); + + installKafkaPlugin(embeddedKafka, queryRunner, topicDescriptions); + + TestingPrestoClient prestoClient = queryRunner.getClient(); + + log.info("Loading data..."); + long startTime = System.nanoTime(); + for (TpchTable table : tables) { + loadTpchTopic(embeddedKafka, prestoClient, table); + } + log.info("Loading complete in %s", nanosSince(startTime).toString(SECONDS)); + + return queryRunner; + } + catch (Throwable e) { + closeAllSuppress(e, queryRunner, embeddedKafka); + throw e; + } + } + + private static void loadTpchTopic(EmbeddedKafka embeddedKafka, TestingPrestoClient prestoClient, TpchTable table) + { + long start = System.nanoTime(); + log.info("Running import for %s", table.getTableName()); + TestUtils.loadTpchTopic(embeddedKafka, prestoClient, kafkaTopicName(table), new QualifiedObjectName("tpch", TINY_SCHEMA_NAME, table.getTableName().toLowerCase(ENGLISH))); + log.info("Imported %s in %s", 0, table.getTableName(), nanosSince(start).convertToMostSuccinctTimeUnit()); + } + + private static String kafkaTopicName(TpchTable table) + { + return TPCH_SCHEMA + "." + table.getTableName().toLowerCase(ENGLISH); + } + + private static Map createTpchTopicDescriptions(Metadata metadata, Iterable> tables) + throws Exception + { + JsonCodec topicDescriptionJsonCodec = new CodecSupplier<>(KafkaTopicDescription.class, metadata).get(); + + ImmutableMap.Builder topicDescriptions = ImmutableMap.builder(); + for (TpchTable table : tables) { + String tableName = table.getTableName(); + SchemaTableName tpchTable = new SchemaTableName(TPCH_SCHEMA, tableName); + + topicDescriptions.put(loadTpchTopicDescription(topicDescriptionJsonCodec, tpchTable.toString(), tpchTable)); + } + return topicDescriptions.build(); + } + + public static Session createSession() + { + return testSessionBuilder() + .setCatalog("kafka") + .setSchema(TPCH_SCHEMA) + .build(); + } + + public static void main(String[] args) + throws Exception + { + Logging.initialize(); + DistributedQueryRunner queryRunner = createKafkaQueryRunner(EmbeddedKafka.createEmbeddedKafka(), TpchTable.getTables()); + Thread.sleep(10); + Logger log = Logger.get(KafkaQueryRunner.class); + log.info("======== SERVER STARTED ========"); + log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl()); + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaConnectorConfig.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaConnectorConfig.java new file mode 100644 index 000000000000..92bc30a9f160 --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaConnectorConfig.java @@ -0,0 +1,62 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.kafka; + +import com.google.common.collect.ImmutableMap; +import io.airlift.configuration.testing.ConfigAssertions; +import org.testng.annotations.Test; + +import java.io.File; +import java.util.Map; + +public class TestKafkaConnectorConfig +{ + @Test + public void testDefaults() + { + ConfigAssertions.assertRecordedDefaults(ConfigAssertions.recordDefaults(KafkaConnectorConfig.class) + .setNodes("") + .setKafkaConnectTimeout("10s") + .setKafkaBufferSize("64kB") + .setDefaultSchema("default") + .setTableNames("") + .setTableDescriptionDir(new File("etc/kafka/")) + .setHideInternalColumns(true)); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = new ImmutableMap.Builder() + .put("kafka.table-description-dir", "/var/lib/kafka") + .put("kafka.table-names", "table1, table2, table3") + .put("kafka.default-schema", "kafka") + .put("kafka.nodes", "localhost:12345,localhost:23456") + .put("kafka.connect-timeout", "1h") + .put("kafka.buffer-size", "1MB") + .put("kafka.hide-internal-columns", "false") + .build(); + + KafkaConnectorConfig expected = new KafkaConnectorConfig() + .setTableDescriptionDir(new File("/var/lib/kafka")) + .setTableNames("table1, table2, table3") + .setDefaultSchema("kafka") + .setNodes("localhost:12345, localhost:23456") + .setKafkaConnectTimeout("1h") + .setKafkaBufferSize("1MB") + .setHideInternalColumns(false); + + ConfigAssertions.assertFullMapping(properties, expected); + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaDistributed.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaDistributed.java new file mode 100644 index 000000000000..c6f25cd98f19 --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaDistributed.java @@ -0,0 +1,49 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.kafka; + +import com.facebook.presto.kafka.util.EmbeddedKafka; +import com.facebook.presto.tests.AbstractTestQueries; +import io.airlift.tpch.TpchTable; +import org.testng.annotations.AfterClass; +import org.testng.annotations.Test; + +import static com.facebook.presto.kafka.KafkaQueryRunner.createKafkaQueryRunner; +import static com.facebook.presto.kafka.util.EmbeddedKafka.createEmbeddedKafka; + +@Test +public class TestKafkaDistributed + extends AbstractTestQueries +{ + private final EmbeddedKafka embeddedKafka; + + public TestKafkaDistributed() + throws Exception + { + this(createEmbeddedKafka()); + } + + public TestKafkaDistributed(EmbeddedKafka embeddedKafka) + throws Exception + { + super(() -> createKafkaQueryRunner(embeddedKafka, TpchTable.getTables())); + this.embeddedKafka = embeddedKafka; + } + + @AfterClass(alwaysRun = true) + public void destroy() + { + embeddedKafka.close(); + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaIntegrationSmokeTest.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaIntegrationSmokeTest.java new file mode 100644 index 000000000000..7d73777d22f1 --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaIntegrationSmokeTest.java @@ -0,0 +1,49 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.kafka; + +import com.facebook.presto.kafka.util.EmbeddedKafka; +import com.facebook.presto.tests.AbstractTestIntegrationSmokeTest; +import org.testng.annotations.AfterClass; +import org.testng.annotations.Test; + +import static com.facebook.presto.kafka.KafkaQueryRunner.createKafkaQueryRunner; +import static com.facebook.presto.kafka.util.EmbeddedKafka.createEmbeddedKafka; +import static io.airlift.tpch.TpchTable.ORDERS; + +@Test +public class TestKafkaIntegrationSmokeTest + extends AbstractTestIntegrationSmokeTest +{ + private final EmbeddedKafka embeddedKafka; + + public TestKafkaIntegrationSmokeTest() + throws Exception + { + this(createEmbeddedKafka()); + } + + public TestKafkaIntegrationSmokeTest(EmbeddedKafka embeddedKafka) + throws Exception + { + super(() -> createKafkaQueryRunner(embeddedKafka, ORDERS)); + this.embeddedKafka = embeddedKafka; + } + + @AfterClass(alwaysRun = true) + public void destroy() + { + embeddedKafka.close(); + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaPlugin.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaPlugin.java new file mode 100644 index 000000000000..ad5e19b0eef4 --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaPlugin.java @@ -0,0 +1,46 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.kafka; + +import com.facebook.presto.spi.connector.Connector; +import com.facebook.presto.spi.connector.ConnectorFactory; +import com.facebook.presto.testing.TestingConnectorContext; +import com.google.common.collect.ImmutableMap; +import org.testng.annotations.Test; + +import static com.google.common.collect.Iterables.getOnlyElement; +import static io.airlift.testing.Assertions.assertInstanceOf; +import static org.testng.Assert.assertNotNull; + +@Test +public class TestKafkaPlugin +{ + @Test + public void testSpinup() + { + KafkaPlugin plugin = new KafkaPlugin(); + + ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); + assertInstanceOf(factory, KafkaConnectorFactory.class); + + Connector c = factory.create( + "test-connector", + ImmutableMap.builder() + .put("kafka.table-names", "test") + .put("kafka.nodes", "localhost:9092") + .build(), + new TestingConnectorContext()); + assertNotNull(c); + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestManySegments.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestManySegments.java new file mode 100644 index 000000000000..4dcba4c146ff --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestManySegments.java @@ -0,0 +1,117 @@ +/* + * 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.kafka; + +import com.facebook.presto.Session; +import com.facebook.presto.kafka.util.EmbeddedKafka; +import com.facebook.presto.kafka.util.TestUtils; +import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.spi.type.BigintType; +import com.facebook.presto.testing.MaterializedResult; +import com.facebook.presto.tests.StandaloneQueryRunner; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import kafka.producer.KeyedMessage; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.util.Properties; +import java.util.UUID; + +import static com.facebook.presto.kafka.util.EmbeddedKafka.CloseableProducer; +import static com.facebook.presto.kafka.util.TestUtils.createEmptyTopicDescription; +import static com.facebook.presto.testing.TestingSession.testSessionBuilder; +import static com.facebook.presto.testing.assertions.Assert.assertEquals; + +@Test(singleThreaded = true) +public class TestManySegments +{ + private static final Session SESSION = testSessionBuilder() + .setCatalog("kafka") + .setSchema("default") + .build(); + + private EmbeddedKafka embeddedKafka; + private String topicName; + private StandaloneQueryRunner queryRunner; + + @BeforeClass + public void startKafka() + throws Exception + { + embeddedKafka = EmbeddedKafka.createEmbeddedKafka(); + embeddedKafka.start(); + + topicName = "test_" + UUID.randomUUID().toString().replaceAll("-", "_"); + + Properties topicProperties = new Properties(); + topicProperties.setProperty("segment.bytes", "1048576"); + + embeddedKafka.createTopics(1, 1, topicProperties, topicName); + + try (CloseableProducer producer = embeddedKafka.createProducer()) { + int jMax = 10_000; + int iMax = 100_000 / jMax; + for (long i = 0; i < iMax; i++) { + ImmutableList.Builder> builder = ImmutableList.builder(); + for (long j = 0; j < jMax; j++) { + builder.add(new KeyedMessage(topicName, i, ImmutableMap.of("id", Long.toString(i * iMax + j), "value", UUID.randomUUID().toString()))); + } + producer.send(builder.build()); + } + } + } + + @AfterClass(alwaysRun = true) + public void stopKafka() + throws Exception + { + embeddedKafka.close(); + } + + @BeforeMethod + public void spinUp() + throws Exception + { + this.queryRunner = new StandaloneQueryRunner(SESSION); + + TestUtils.installKafkaPlugin(embeddedKafka, queryRunner, + ImmutableMap.builder() + .put(createEmptyTopicDescription(topicName, new SchemaTableName("default", topicName))) + .build()); + } + + @AfterMethod(alwaysRun = true) + public void tearDown() + throws Exception + { + queryRunner.close(); + } + + @Test + public void testManySegments() + throws Exception + { + MaterializedResult result = queryRunner.execute("SELECT count(_message) from " + topicName); + + MaterializedResult expected = MaterializedResult.resultBuilder(SESSION, BigintType.BIGINT) + .row(100000L) + .build(); + + assertEquals(result, expected); + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestMinimalFunctionality.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestMinimalFunctionality.java new file mode 100644 index 000000000000..1c6b3949e9b8 --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestMinimalFunctionality.java @@ -0,0 +1,143 @@ +/* + * 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.kafka; + +import com.facebook.presto.Session; +import com.facebook.presto.kafka.util.EmbeddedKafka; +import com.facebook.presto.kafka.util.TestUtils; +import com.facebook.presto.metadata.QualifiedObjectName; +import com.facebook.presto.metadata.TableHandle; +import com.facebook.presto.security.AllowAllAccessControl; +import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.spi.type.BigintType; +import com.facebook.presto.testing.MaterializedResult; +import com.facebook.presto.tests.StandaloneQueryRunner; +import com.google.common.collect.ImmutableMap; +import kafka.producer.KeyedMessage; +import org.testng.annotations.AfterClass; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.util.Optional; +import java.util.Properties; +import java.util.UUID; + +import static com.facebook.presto.kafka.util.EmbeddedKafka.CloseableProducer; +import static com.facebook.presto.kafka.util.TestUtils.createEmptyTopicDescription; +import static com.facebook.presto.testing.TestingSession.testSessionBuilder; +import static com.facebook.presto.testing.assertions.Assert.assertEquals; +import static com.facebook.presto.transaction.TransactionBuilder.transaction; +import static org.testng.Assert.assertTrue; + +@Test(singleThreaded = true) +public class TestMinimalFunctionality +{ + private static final Session SESSION = testSessionBuilder() + .setCatalog("kafka") + .setSchema("default") + .build(); + + private EmbeddedKafka embeddedKafka; + private String topicName; + private StandaloneQueryRunner queryRunner; + + @BeforeClass + public void startKafka() + throws Exception + { + embeddedKafka = EmbeddedKafka.createEmbeddedKafka(); + embeddedKafka.start(); + } + + @AfterClass + public void stopKafka() + throws Exception + { + embeddedKafka.close(); + } + + @BeforeMethod + public void spinUp() + throws Exception + { + this.topicName = "test_" + UUID.randomUUID().toString().replaceAll("-", "_"); + + Properties topicProperties = new Properties(); + embeddedKafka.createTopics(2, 1, topicProperties, topicName); + + this.queryRunner = new StandaloneQueryRunner(SESSION); + + TestUtils.installKafkaPlugin(embeddedKafka, queryRunner, + ImmutableMap.builder() + .put(createEmptyTopicDescription(topicName, new SchemaTableName("default", topicName))) + .build()); + } + + @AfterMethod + public void tearDown() + throws Exception + { + queryRunner.close(); + } + + private void createMessages(String topicName, int count) + { + try (CloseableProducer producer = embeddedKafka.createProducer()) { + for (long i = 0; i < count; i++) { + Object message = ImmutableMap.of("id", Long.toString(i), "value", UUID.randomUUID().toString()); + producer.send(new KeyedMessage<>(topicName, i, message)); + } + } + } + + @Test + public void testTopicExists() + throws Exception + { + QualifiedObjectName name = new QualifiedObjectName("kafka", "default", topicName); + + transaction(queryRunner.getTransactionManager(), new AllowAllAccessControl()) + .singleStatement() + .execute(SESSION, session -> { + Optional handle = queryRunner.getServer().getMetadata().getTableHandle(session, name); + assertTrue(handle.isPresent()); + }); + } + + @Test + public void testTopicHasData() + throws Exception + { + MaterializedResult result = queryRunner.execute("SELECT count(1) from " + topicName); + + MaterializedResult expected = MaterializedResult.resultBuilder(SESSION, BigintType.BIGINT) + .row(0L) + .build(); + + assertEquals(result, expected); + + int count = 1000; + createMessages(topicName, count); + + result = queryRunner.execute("SELECT count(1) from " + topicName); + + expected = MaterializedResult.resultBuilder(SESSION, BigintType.BIGINT) + .row((long) count) + .build(); + + assertEquals(result, expected); + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/CodecSupplier.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/CodecSupplier.java new file mode 100644 index 000000000000..05f7a035869f --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/CodecSupplier.java @@ -0,0 +1,71 @@ +/* + * 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.kafka.util; + +import com.facebook.presto.metadata.Metadata; +import com.facebook.presto.spi.type.Type; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.deser.std.FromStringDeserializer; +import com.google.common.collect.ImmutableMap; +import io.airlift.json.JsonCodec; +import io.airlift.json.JsonCodecFactory; +import io.airlift.json.ObjectMapperProvider; + +import java.util.function.Supplier; + +import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature; + +public final class CodecSupplier + implements Supplier> +{ + private final Metadata metadata; + private final JsonCodecFactory codecFactory; + private final Class clazz; + + public CodecSupplier(Class clazz, Metadata metadata) + { + this.clazz = clazz; + this.metadata = metadata; + ObjectMapperProvider objectMapperProvider = new ObjectMapperProvider(); + objectMapperProvider.setJsonDeserializers(ImmutableMap.of(Type.class, new TypeDeserializer())); + this.codecFactory = new JsonCodecFactory(objectMapperProvider); + } + + @Override + public JsonCodec get() + { + return codecFactory.jsonCodec(clazz); + } + + private class TypeDeserializer + extends FromStringDeserializer + { + private static final long serialVersionUID = 1L; + + public TypeDeserializer() + { + super(Type.class); + } + + @Override + protected Type _deserialize(String value, DeserializationContext context) + { + Type type = metadata.getType(parseTypeSignature(value)); + if (type == null) { + throw new IllegalArgumentException(String.valueOf("Unknown type " + value)); + } + return type; + } + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedKafka.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedKafka.java new file mode 100644 index 000000000000..f0db0e309618 --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedKafka.java @@ -0,0 +1,175 @@ +/* + * 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.kafka.util; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; +import com.google.common.io.Files; +import kafka.admin.AdminUtils; +import kafka.javaapi.producer.Producer; +import kafka.producer.ProducerConfig; +import kafka.server.KafkaConfig; +import kafka.server.KafkaServerStartable; +import kafka.utils.ZKStringSerializer$; +import org.I0Itec.zkclient.ZkClient; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; + +import static com.facebook.presto.kafka.util.TestUtils.findUnusedPort; +import static com.facebook.presto.kafka.util.TestUtils.toProperties; +import static com.google.common.base.Preconditions.checkState; +import static io.airlift.testing.FileUtils.deleteRecursively; +import static java.util.Objects.requireNonNull; + +public class EmbeddedKafka + implements Closeable +{ + private final EmbeddedZookeeper zookeeper; + private final int port; + private final File kafkaDataDir; + private final KafkaServerStartable kafka; + + private final AtomicBoolean started = new AtomicBoolean(); + private final AtomicBoolean stopped = new AtomicBoolean(); + + public static EmbeddedKafka createEmbeddedKafka() + throws IOException + { + return new EmbeddedKafka(new EmbeddedZookeeper(), new Properties()); + } + + public static EmbeddedKafka createEmbeddedKafka(Properties overrideProperties) + throws IOException + { + return new EmbeddedKafka(new EmbeddedZookeeper(), overrideProperties); + } + + EmbeddedKafka(EmbeddedZookeeper zookeeper, Properties overrideProperties) + throws IOException + { + this.zookeeper = requireNonNull(zookeeper, "zookeeper is null"); + requireNonNull(overrideProperties, "overrideProperties is null"); + + this.port = findUnusedPort(); + this.kafkaDataDir = Files.createTempDir(); + + Map properties = ImmutableMap.builder() + .put("broker.id", "0") + .put("host.name", "localhost") + .put("num.partitions", "2") + .put("log.flush.interval.messages", "10000") + .put("log.flush.interval.ms", "1000") + .put("log.retention.minutes", "60") + .put("log.segment.bytes", "1048576") + .put("auto.create.topics.enable", "false") + .put("zookeeper.connection.timeout.ms", "1000000") + .put("port", Integer.toString(port)) + .put("log.dirs", kafkaDataDir.getAbsolutePath()) + .put("zookeeper.connect", zookeeper.getConnectString()) + .putAll(Maps.fromProperties(overrideProperties)) + .build(); + + KafkaConfig config = new KafkaConfig(toProperties(properties)); + this.kafka = new KafkaServerStartable(config); + } + + public void start() + throws InterruptedException, IOException + { + if (!started.getAndSet(true)) { + zookeeper.start(); + kafka.startup(); + } + } + + @Override + public void close() + { + if (started.get() && !stopped.getAndSet(true)) { + kafka.shutdown(); + kafka.awaitShutdown(); + zookeeper.close(); + deleteRecursively(kafkaDataDir); + } + } + + public void createTopics(String... topics) + { + createTopics(2, 1, new Properties(), topics); + } + + public void createTopics(int partitions, int replication, Properties topicProperties, String... topics) + { + checkState(started.get() && !stopped.get(), "not started!"); + + ZkClient zkClient = new ZkClient(getZookeeperConnectString(), 30_000, 30_000, ZKStringSerializer$.MODULE$); + try { + for (String topic : topics) { + AdminUtils.createTopic(zkClient, topic, partitions, replication, topicProperties); + } + } + finally { + zkClient.close(); + } + } + + public CloseableProducer createProducer() + { + Map properties = ImmutableMap.builder() + .put("metadata.broker.list", getConnectString()) + .put("serializer.class", JsonEncoder.class.getName()) + .put("key.serializer.class", NumberEncoder.class.getName()) + .put("partitioner.class", NumberPartitioner.class.getName()) + .put("request.required.acks", "1") + .build(); + + ProducerConfig producerConfig = new ProducerConfig(toProperties(properties)); + return new CloseableProducer<>(producerConfig); + } + + public static class CloseableProducer + extends Producer + implements AutoCloseable + { + public CloseableProducer(ProducerConfig config) + { + super(config); + } + } + + public int getZookeeperPort() + { + return zookeeper.getPort(); + } + + public int getPort() + { + return port; + } + + public String getConnectString() + { + return "localhost:" + Integer.toString(port); + } + + public String getZookeeperConnectString() + { + return zookeeper.getConnectString(); + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedZookeeper.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedZookeeper.java new file mode 100644 index 000000000000..e5a77f863b42 --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedZookeeper.java @@ -0,0 +1,97 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.kafka.util; + +import com.google.common.io.Files; +import org.apache.zookeeper.server.NIOServerCnxnFactory; +import org.apache.zookeeper.server.ServerCnxnFactory; +import org.apache.zookeeper.server.ZooKeeperServer; +import org.apache.zookeeper.server.persistence.FileTxnSnapLog; + +import java.io.Closeable; +import java.io.File; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.concurrent.atomic.AtomicBoolean; + +import static io.airlift.testing.FileUtils.deleteRecursively; + +public class EmbeddedZookeeper + implements Closeable +{ + private final int port; + private final File zkDataDir; + private final ZooKeeperServer zkServer; + private final ServerCnxnFactory cnxnFactory; + + private final AtomicBoolean started = new AtomicBoolean(); + private final AtomicBoolean stopped = new AtomicBoolean(); + + public EmbeddedZookeeper() + throws IOException + { + this(TestUtils.findUnusedPort()); + } + + public EmbeddedZookeeper(int port) + throws IOException + { + this.port = port; + zkDataDir = Files.createTempDir(); + zkServer = new ZooKeeperServer(); + + FileTxnSnapLog ftxn = new FileTxnSnapLog(zkDataDir, zkDataDir); + zkServer.setTxnLogFactory(ftxn); + + cnxnFactory = NIOServerCnxnFactory.createFactory(new InetSocketAddress(port), 0); + } + + public void start() + throws InterruptedException, IOException + { + if (!started.getAndSet(true)) { + cnxnFactory.startup(zkServer); + } + } + + @Override + public void close() + { + if (started.get() && !stopped.getAndSet(true)) { + cnxnFactory.shutdown(); + try { + cnxnFactory.join(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + + if (zkServer.isRunning()) { + zkServer.shutdown(); + } + + deleteRecursively(zkDataDir); + } + } + + public String getConnectString() + { + return "127.0.0.1:" + Integer.toString(port); + } + + public int getPort() + { + return port; + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/JsonEncoder.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/JsonEncoder.java new file mode 100644 index 000000000000..292b184daedb --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/JsonEncoder.java @@ -0,0 +1,44 @@ +/* + * 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.kafka.util; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; +import kafka.serializer.Encoder; +import kafka.utils.VerifiableProperties; + +import java.io.IOException; + +public class JsonEncoder + implements Encoder +{ + private final ObjectMapper objectMapper = new ObjectMapper(); + + @SuppressWarnings("UnusedParameters") + public JsonEncoder(VerifiableProperties properties) + { + // constructor required by Kafka + } + + @Override + public byte[] toBytes(Object o) + { + try { + return objectMapper.writeValueAsBytes(o); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/KafkaLoader.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/KafkaLoader.java new file mode 100644 index 000000000000..259fdd7aca53 --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/KafkaLoader.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.kafka.util; + +import com.facebook.presto.Session; +import com.facebook.presto.client.Column; +import com.facebook.presto.client.QueryResults; +import com.facebook.presto.server.testing.TestingPrestoServer; +import com.facebook.presto.spi.type.TimeZoneKey; +import com.facebook.presto.spi.type.Type; +import com.facebook.presto.spi.type.Varchars; +import com.facebook.presto.tests.AbstractTestingPrestoClient; +import com.facebook.presto.tests.ResultsSession; +import com.google.common.collect.ImmutableMap; +import kafka.javaapi.producer.Producer; +import kafka.producer.KeyedMessage; +import org.joda.time.format.DateTimeFormatter; +import org.joda.time.format.ISODateTimeFormat; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +import static com.facebook.presto.spi.type.BigintType.BIGINT; +import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; +import static com.facebook.presto.spi.type.DateTimeEncoding.unpackMillisUtc; +import static com.facebook.presto.spi.type.DateType.DATE; +import static com.facebook.presto.spi.type.DoubleType.DOUBLE; +import static com.facebook.presto.spi.type.IntegerType.INTEGER; +import static com.facebook.presto.spi.type.TimeType.TIME; +import static com.facebook.presto.spi.type.TimeWithTimeZoneType.TIME_WITH_TIME_ZONE; +import static com.facebook.presto.spi.type.TimestampType.TIMESTAMP; +import static com.facebook.presto.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; +import static com.facebook.presto.util.DateTimeUtils.parseTime; +import static com.facebook.presto.util.DateTimeUtils.parseTimestampWithTimeZone; +import static com.facebook.presto.util.DateTimeUtils.parseTimestampWithoutTimeZone; +import static com.google.common.base.Preconditions.checkState; +import static java.util.Objects.requireNonNull; + +public class KafkaLoader + extends AbstractTestingPrestoClient +{ + private static final DateTimeFormatter ISO8601_FORMATTER = ISODateTimeFormat.dateTime(); + + private final String topicName; + private final Producer producer; + private final AtomicLong count = new AtomicLong(); + + public KafkaLoader(Producer producer, + String topicName, + TestingPrestoServer prestoServer, + Session defaultSession) + { + super(prestoServer, defaultSession); + + this.topicName = topicName; + this.producer = producer; + } + + @Override + public ResultsSession getResultSession(Session session) + { + requireNonNull(session, "session is null"); + return new KafkaLoadingSession(session); + } + + private class KafkaLoadingSession + implements ResultsSession + { + private final AtomicReference> types = new AtomicReference<>(); + + private final TimeZoneKey timeZoneKey; + + private KafkaLoadingSession(Session session) + { + this.timeZoneKey = session.getTimeZoneKey(); + } + + @Override + public void addResults(QueryResults results) + { + if (types.get() == null && results.getColumns() != null) { + types.set(getTypes(results.getColumns())); + } + + if (results.getData() != null) { + checkState(types.get() != null, "Data without types received!"); + List columns = results.getColumns(); + for (List fields : results.getData()) { + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (int i = 0; i < fields.size(); i++) { + Type type = types.get().get(i); + Object value = convertValue(fields.get(i), type); + if (value != null) { + builder.put(columns.get(i).getName(), value); + } + } + + producer.send(new KeyedMessage<>(topicName, count.getAndIncrement(), builder.build())); + } + } + } + + @Override + public Void build(Map setSessionProperties, Set resetSessionProperties) + { + return null; + } + + private Object convertValue(Object value, Type type) + { + if (value == null) { + return null; + } + + if (BOOLEAN.equals(type) || Varchars.isVarcharType(type)) { + return value; + } + if (BIGINT.equals(type)) { + return ((Number) value).longValue(); + } + if (INTEGER.equals(type)) { + return ((Number) value).intValue(); + } + if (DOUBLE.equals(type)) { + return ((Number) value).doubleValue(); + } + if (DATE.equals(type)) { + return value; + } + if (TIME.equals(type)) { + return ISO8601_FORMATTER.print(parseTime(timeZoneKey, (String) value)); + } + if (TIMESTAMP.equals(type)) { + return ISO8601_FORMATTER.print(parseTimestampWithoutTimeZone(timeZoneKey, (String) value)); + } + if (TIME_WITH_TIME_ZONE.equals(type) || TIMESTAMP_WITH_TIME_ZONE.equals(type)) { + return ISO8601_FORMATTER.print(unpackMillisUtc(parseTimestampWithTimeZone(timeZoneKey, (String) value))); + } + throw new AssertionError("unhandled type: " + type); + } + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberEncoder.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberEncoder.java new file mode 100644 index 000000000000..e0708a5c15d1 --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberEncoder.java @@ -0,0 +1,37 @@ +/* + * 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.kafka.util; + +import kafka.serializer.Encoder; +import kafka.utils.VerifiableProperties; + +import java.nio.ByteBuffer; + +public class NumberEncoder + implements Encoder +{ + @SuppressWarnings("UnusedParameters") + public NumberEncoder(VerifiableProperties properties) + { + // constructor required by Kafka + } + + @Override + public byte[] toBytes(Number value) + { + ByteBuffer buf = ByteBuffer.allocate(8); + buf.putLong(value == null ? 0L : value.longValue()); + return buf.array(); + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberPartitioner.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberPartitioner.java new file mode 100644 index 000000000000..d8b5aa3f6eba --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberPartitioner.java @@ -0,0 +1,38 @@ +/* + * 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.kafka.util; + +import kafka.producer.Partitioner; +import kafka.utils.VerifiableProperties; + +import static java.lang.Math.toIntExact; + +public class NumberPartitioner + implements Partitioner +{ + @SuppressWarnings("UnusedParameters") + public NumberPartitioner(VerifiableProperties properties) + { + // constructor required by Kafka + } + + @Override + public int partition(Object key, int numPartitions) + { + if (key instanceof Number) { + return toIntExact(((Number) key).longValue() % numPartitions); + } + return 0; + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/TestUtils.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/TestUtils.java new file mode 100644 index 000000000000..75090a7ab0df --- /dev/null +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/TestUtils.java @@ -0,0 +1,95 @@ +/* + * 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.kafka.util; + +import com.facebook.presto.kafka.KafkaPlugin; +import com.facebook.presto.kafka.KafkaTopicDescription; +import com.facebook.presto.metadata.QualifiedObjectName; +import com.facebook.presto.spi.SchemaTableName; +import com.facebook.presto.testing.QueryRunner; +import com.facebook.presto.tests.TestingPrestoClient; +import com.google.common.base.Joiner; +import com.google.common.collect.ImmutableMap; +import com.google.common.io.ByteStreams; +import io.airlift.json.JsonCodec; + +import java.io.IOException; +import java.net.ServerSocket; +import java.util.AbstractMap; +import java.util.Map; +import java.util.Properties; + +import static com.facebook.presto.kafka.util.EmbeddedKafka.CloseableProducer; +import static java.lang.String.format; + +public final class TestUtils +{ + private TestUtils() {} + + public static int findUnusedPort() + throws IOException + { + try (ServerSocket socket = new ServerSocket(0)) { + return socket.getLocalPort(); + } + } + + public static Properties toProperties(Map map) + { + Properties properties = new Properties(); + for (Map.Entry entry : map.entrySet()) { + properties.setProperty(entry.getKey(), entry.getValue()); + } + return properties; + } + + public static void installKafkaPlugin(EmbeddedKafka embeddedKafka, QueryRunner queryRunner, Map topicDescriptions) + { + KafkaPlugin kafkaPlugin = new KafkaPlugin(); + kafkaPlugin.setTableDescriptionSupplier(() -> topicDescriptions); + queryRunner.installPlugin(kafkaPlugin); + + Map kafkaConfig = ImmutableMap.of( + "kafka.nodes", embeddedKafka.getConnectString(), + "kafka.table-names", Joiner.on(",").join(topicDescriptions.keySet()), + "kafka.connect-timeout", "120s", + "kafka.default-schema", "default"); + queryRunner.createCatalog("kafka", "kafka", kafkaConfig); + } + + public static void loadTpchTopic(EmbeddedKafka embeddedKafka, TestingPrestoClient prestoClient, String topicName, QualifiedObjectName tpchTableName) + { + try (CloseableProducer producer = embeddedKafka.createProducer(); + KafkaLoader tpchLoader = new KafkaLoader(producer, topicName, prestoClient.getServer(), prestoClient.getDefaultSession())) { + tpchLoader.execute(format("SELECT * from %s", tpchTableName)); + } + } + + public static Map.Entry loadTpchTopicDescription(JsonCodec topicDescriptionJsonCodec, String topicName, SchemaTableName schemaTableName) + throws IOException + { + KafkaTopicDescription tpchTemplate = topicDescriptionJsonCodec.fromJson(ByteStreams.toByteArray(TestUtils.class.getResourceAsStream(format("/tpch/%s.json", schemaTableName.getTableName())))); + + return new AbstractMap.SimpleImmutableEntry<>( + schemaTableName, + new KafkaTopicDescription(schemaTableName.getTableName(), schemaTableName.getSchemaName(), topicName, tpchTemplate.getKey(), tpchTemplate.getMessage())); + } + + public static Map.Entry createEmptyTopicDescription(String topicName, SchemaTableName schemaTableName) + { + return new AbstractMap.SimpleImmutableEntry<>( + schemaTableName, + new KafkaTopicDescription(schemaTableName.getTableName(), schemaTableName.getSchemaName(), topicName, null, null)); + } +} diff --git a/presto-kafka07/src/test/resources/tpch/customer.json b/presto-kafka07/src/test/resources/tpch/customer.json new file mode 100644 index 000000000000..cf800fc0a9bb --- /dev/null +++ b/presto-kafka07/src/test/resources/tpch/customer.json @@ -0,0 +1,61 @@ +{ + "tableName": "customer", + "schemaName": "tpch", + "topicName": "tpch.customer", + "key": { + "dataFormat": "raw", + "fields": [ + { + "name": "kafka_key", + "dataFormat": "LONG", + "type": "BIGINT", + "hidden": "true" + } + ] + }, + "message": { + "dataFormat": "json", + "fields": [ + { + "name": "custkey", + "mapping": "custkey", + "type": "BIGINT" + }, + { + "name": "name", + "mapping": "name", + "type": "VARCHAR(25)" + }, + { + "name": "address", + "mapping": "address", + "type": "VARCHAR(40)" + }, + { + "name": "nationkey", + "mapping": "nationkey", + "type": "BIGINT" + }, + { + "name": "phone", + "mapping": "phone", + "type": "VARCHAR(15)" + }, + { + "name": "acctbal", + "mapping": "acctbal", + "type": "DOUBLE" + }, + { + "name": "mktsegment", + "mapping": "mktsegment", + "type": "VARCHAR(10)" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(117)" + } + ] + } +} diff --git a/presto-kafka07/src/test/resources/tpch/lineitem.json b/presto-kafka07/src/test/resources/tpch/lineitem.json new file mode 100644 index 000000000000..2250fddf7771 --- /dev/null +++ b/presto-kafka07/src/test/resources/tpch/lineitem.json @@ -0,0 +1,104 @@ +{ + "tableName": "lineitem", + "schemaName": "tpch", + "topicName": "tpch.lineitem", + "key": { + "dataFormat": "raw", + "fields": [ + { + "name": "kafka_key", + "dataFormat": "LONG", + "type": "BIGINT", + "hidden": "true" + } + ] + }, + "message": { + "dataFormat": "json", + "fields": [ + { + "name": "orderkey", + "mapping": "orderkey", + "type": "BIGINT" + }, + { + "name": "partkey", + "mapping": "partkey", + "type": "BIGINT" + }, + { + "name": "suppkey", + "mapping": "suppkey", + "type": "BIGINT" + }, + { + "name": "linenumber", + "mapping": "linenumber", + "type": "INTEGER" + }, + { + "name": "quantity", + "mapping": "quantity", + "type": "DOUBLE" + }, + { + "name": "extendedprice", + "mapping": "extendedprice", + "type": "DOUBLE" + }, + { + "name": "discount", + "mapping": "discount", + "type": "DOUBLE" + }, + { + "name": "tax", + "mapping": "tax", + "type": "DOUBLE" + }, + { + "name": "returnflag", + "mapping": "returnflag", + "type": "VARCHAR(1)" + }, + { + "name": "linestatus", + "mapping": "linestatus", + "type": "VARCHAR(1)" + }, + { + "name": "shipdate", + "mapping": "shipdate", + "type": "DATE", + "dataFormat": "iso8601" + }, + { + "name": "commitdate", + "mapping": "commitdate", + "type": "DATE", + "dataFormat": "iso8601" + }, + { + "name": "receiptdate", + "mapping": "receiptdate", + "type": "DATE", + "dataFormat": "iso8601" + }, + { + "name": "shipinstruct", + "mapping": "shipinstruct", + "type": "VARCHAR(25)" + }, + { + "name": "shipmode", + "mapping": "shipmode", + "type": "VARCHAR(7)" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(44)" + } + ] + } +} diff --git a/presto-kafka07/src/test/resources/tpch/nation.json b/presto-kafka07/src/test/resources/tpch/nation.json new file mode 100644 index 000000000000..0749a1249d76 --- /dev/null +++ b/presto-kafka07/src/test/resources/tpch/nation.json @@ -0,0 +1,41 @@ +{ + "tableName": "nation", + "schemaName": "tpch", + "topicName": "tpch.nation", + "key": { + "dataFormat": "raw", + "fields": [ + { + "name": "kafka_key", + "dataFormat": "LONG", + "type": "BIGINT", + "hidden": "true" + } + ] + }, + "message": { + "dataFormat": "json", + "fields": [ + { + "name": "nationkey", + "mapping": "nationkey", + "type": "BIGINT" + }, + { + "name": "name", + "mapping": "name", + "type": "VARCHAR(25)" + }, + { + "name": "regionkey", + "mapping": "regionkey", + "type": "BIGINT" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(152)" + } + ] + } +} diff --git a/presto-kafka07/src/test/resources/tpch/orders.json b/presto-kafka07/src/test/resources/tpch/orders.json new file mode 100644 index 000000000000..f5eb594083a4 --- /dev/null +++ b/presto-kafka07/src/test/resources/tpch/orders.json @@ -0,0 +1,67 @@ +{ + "tableName": "orders", + "schemaName": "tpch", + "topicName": "tpch.orders", + "key": { + "dataFormat": "raw", + "fields": [ + { + "name": "kafka_key", + "dataFormat": "LONG", + "type": "BIGINT", + "hidden": "true" + } + ] + }, + "message": { + "dataFormat": "json", + "fields": [ + { + "name": "orderkey", + "mapping": "orderkey", + "type": "BIGINT" + }, + { + "name": "custkey", + "mapping": "custkey", + "type": "BIGINT" + }, + { + "name": "orderstatus", + "mapping": "orderstatus", + "type": "VARCHAR(1)" + }, + { + "name": "totalprice", + "mapping": "totalprice", + "type": "DOUBLE" + }, + { + "name": "orderdate", + "mapping": "orderdate", + "type": "DATE", + "dataFormat": "iso8601" + }, + { + "name": "orderpriority", + "mapping": "orderpriority", + "type": "VARCHAR(15)" + }, + { + "name": "clerk", + "mapping": "clerk", + "type": "VARCHAR(15)" + }, + { + "name": "shippriority", + "mapping": "shippriority", + "type": "INTEGER" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(79)" + } + ] + } +} diff --git a/presto-kafka07/src/test/resources/tpch/part.json b/presto-kafka07/src/test/resources/tpch/part.json new file mode 100644 index 000000000000..3c56b27955f1 --- /dev/null +++ b/presto-kafka07/src/test/resources/tpch/part.json @@ -0,0 +1,66 @@ +{ + "tableName": "part", + "schemaName": "tpch", + "topicName": "tpch.part", + "key": { + "dataFormat": "raw", + "fields": [ + { + "name": "kafka_key", + "dataFormat": "LONG", + "type": "BIGINT", + "hidden": "true" + } + ] + }, + "message": { + "dataFormat": "json", + "fields": [ + { + "name": "partkey", + "mapping": "partkey", + "type": "BIGINT" + }, + { + "name": "name", + "mapping": "name", + "type": "VARCHAR(55)" + }, + { + "name": "mfgr", + "mapping": "mfgr", + "type": "VARCHAR(25)" + }, + { + "name": "brand", + "mapping": "brand", + "type": "VARCHAR(10)" + }, + { + "name": "type", + "mapping": "type", + "type": "VARCHAR(25)" + }, + { + "name": "size", + "mapping": "size", + "type": "INTEGER" + }, + { + "name": "container", + "mapping": "container", + "type": "VARCHAR(10)" + }, + { + "name": "retailprice", + "mapping": "retailprice", + "type": "DOUBLE" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(23)" + } + ] + } +} diff --git a/presto-kafka07/src/test/resources/tpch/partsupp.json b/presto-kafka07/src/test/resources/tpch/partsupp.json new file mode 100644 index 000000000000..cd6d6364f87f --- /dev/null +++ b/presto-kafka07/src/test/resources/tpch/partsupp.json @@ -0,0 +1,46 @@ +{ + "tableName": "partsupp", + "schemaName": "tpch", + "topicName": "tpch.partsupp", + "key": { + "dataFormat": "raw", + "fields": [ + { + "name": "kafka_key", + "dataFormat": "LONG", + "type": "BIGINT", + "hidden": "true" + } + ] + }, + "message": { + "dataFormat": "json", + "fields": [ + { + "name": "partkey", + "mapping": "partkey", + "type": "BIGINT" + }, + { + "name": "suppkey", + "mapping": "suppkey", + "type": "BIGINT" + }, + { + "name": "availqty", + "mapping": "availqty", + "type": "INTEGER" + }, + { + "name": "supplycost", + "mapping": "supplycost", + "type": "DOUBLE" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(199)" + } + ] + } +} diff --git a/presto-kafka07/src/test/resources/tpch/region.json b/presto-kafka07/src/test/resources/tpch/region.json new file mode 100644 index 000000000000..4aca016e22cc --- /dev/null +++ b/presto-kafka07/src/test/resources/tpch/region.json @@ -0,0 +1,36 @@ +{ + "tableName": "region", + "schemaName": "tpch", + "topicName": "tpch.region", + "key": { + "dataFormat": "raw", + "fields": [ + { + "name": "kafka_key", + "dataFormat": "LONG", + "type": "BIGINT", + "hidden": "true" + } + ] + }, + "message": { + "dataFormat": "json", + "fields": [ + { + "name": "regionkey", + "mapping": "regionkey", + "type": "BIGINT" + }, + { + "name": "name", + "mapping": "name", + "type": "VARCHAR(25)" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(152)" + } + ] + } +} diff --git a/presto-kafka07/src/test/resources/tpch/supplier.json b/presto-kafka07/src/test/resources/tpch/supplier.json new file mode 100644 index 000000000000..af9d24c86a83 --- /dev/null +++ b/presto-kafka07/src/test/resources/tpch/supplier.json @@ -0,0 +1,56 @@ +{ + "tableName": "supplier", + "schemaName": "tpch", + "topicName": "tpch.supplier", + "key": { + "dataFormat": "raw", + "fields": [ + { + "name": "kafka_key", + "dataFormat": "LONG", + "type": "BIGINT", + "hidden": "true" + } + ] + }, + "message": { + "dataFormat": "json", + "fields": [ + { + "name": "suppkey", + "mapping": "suppkey", + "type": "BIGINT" + }, + { + "name": "name", + "mapping": "name", + "type": "VARCHAR(25)" + }, + { + "name": "address", + "mapping": "address", + "type": "VARCHAR(40)" + }, + { + "name": "nationkey", + "mapping": "nationkey", + "type": "BIGINT" + }, + { + "name": "phone", + "mapping": "phone", + "type": "VARCHAR(15)" + }, + { + "name": "acctbal", + "mapping": "acctbal", + "type": "DOUBLE" + }, + { + "name": "comment", + "mapping": "comment", + "type": "VARCHAR(101)" + } + ] + } +} diff --git a/presto-main/etc/config.properties b/presto-main/etc/config.properties index 44d7c148de4e..fc6ecda74421 100644 --- a/presto-main/etc/config.properties +++ b/presto-main/etc/config.properties @@ -34,6 +34,7 @@ plugin.bundles=\ ../presto-hive-hadoop2/pom.xml,\ ../presto-example-http/pom.xml,\ ../presto-kafka/pom.xml, \ + ../presto-kafka07/pom.xml, \ ../presto-tpch/pom.xml, \ ../presto-local-file/pom.xml, \ ../presto-mysql/pom.xml,\ diff --git a/presto-server/src/main/provisio/presto.xml b/presto-server/src/main/provisio/presto.xml index 387f9c4c247f..7ff4a8b506bc 100644 --- a/presto-server/src/main/provisio/presto.xml +++ b/presto-server/src/main/provisio/presto.xml @@ -74,6 +74,12 @@ + + + + + + From 47ac2526740378670b3f66bc7dd7434caad2029a Mon Sep 17 00:00:00 2001 From: Xiaoyao Qian Date: Thu, 20 Jul 2017 11:31:09 -0700 Subject: [PATCH 212/331] kafka07 changes --- presto-kafka07/pom.xml | 56 ++++-- .../presto/kafka/KafkaConnectorConfig.java | 39 ++++- .../presto/kafka/KafkaConnectorFactory.java | 2 +- .../kafka/KafkaInternalFieldDescription.java | 7 +- .../facebook/presto/kafka/KafkaMetadata.java | 33 +++- .../facebook/presto/kafka/KafkaRecordSet.java | 126 ++++++++------ .../presto/kafka/KafkaRecordSetProvider.java | 6 +- .../kafka/KafkaSimpleConsumerManager.java | 6 +- .../com/facebook/presto/kafka/KafkaSplit.java | 20 ++- .../presto/kafka/KafkaSplitManager.java | 145 ++++++++-------- .../presto/kafka/KafkaTableLayoutHandle.java | 22 ++- .../com/facebook/presto/kafka/KafkaUtil.java | 159 ++++++++++++++++++ .../kafka/TestKafkaConnectorConfig.java | 11 +- .../presto/kafka/TestManySegments.java | 7 +- .../kafka/TestMinimalFunctionality.java | 10 +- .../presto/kafka/util/EmbeddedKafka.java | 39 ++--- .../presto/kafka/util/EmbeddedZookeeper.java | 4 +- .../presto/kafka/util/JsonEncoder.java | 13 +- .../presto/kafka/util/KafkaLoader.java | 5 +- .../presto/kafka/util/NumberEncoder.java | 12 +- .../presto/kafka/util/NumberPartitioner.java | 16 +- .../facebook/presto/kafka/util/TestUtils.java | 4 +- 22 files changed, 533 insertions(+), 209 deletions(-) create mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaUtil.java diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index ffcb91138731..ef33044e62ad 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -14,6 +14,9 @@ ${project.parent.basedir} + + + true @@ -63,8 +66,33 @@ - org.apache.kafka - kafka_2.10 + com.twitter + rosette-kafka_2.11 + 0.7.2-21 + + + jsr305 + com.google.code.findbugs + + + zookeeper + org.apache.zookeeper + + + finagle-ostrich4_2.10 + com.twitter + + + commons-lang + commons-lang + + + + + + org.apache.zookeeper + zookeeper + @@ -72,6 +100,12 @@ joda-time + + org.scala-lang + scala-library + 2.11.7 + + javax.annotation javax.annotation-api @@ -163,17 +197,15 @@ - org.apache.zookeeper - zookeeper - - 3.4.9 - test - - - - com.101tec + com.github.sgroschupf zkclient - test + 0.1 + + + log4j + log4j + + diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorConfig.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorConfig.java index ae5671f6512e..de5615de2167 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorConfig.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorConfig.java @@ -23,7 +23,6 @@ import io.airlift.units.MinDuration; import javax.validation.constraints.NotNull; -import javax.validation.constraints.Size; import java.io.File; import java.util.Set; @@ -62,13 +61,23 @@ public class KafkaConnectorConfig /** * Folder holding the JSON description files for Kafka topics. */ - private File tableDescriptionDir = new File("etc/kafka/"); + private File tableDescriptionDir = new File("etc/kafka07/"); /** * Whether internal columns are shown in table metadata or not. Default is no. */ private boolean hideInternalColumns = true; + /** + * ZK endpoint for getting broker list + */ + private String zkEndpoint = ""; + + /** + * Fetch size + */ + private int fetchSize = 10 * 1024 * 1024; + @NotNull public File getTableDescriptionDir() { @@ -108,7 +117,6 @@ public KafkaConnectorConfig setDefaultSchema(String defaultSchema) return this; } - @Size(min = 1) public Set getNodes() { return nodes; @@ -158,6 +166,31 @@ public KafkaConnectorConfig setHideInternalColumns(boolean hideInternalColumns) return this; } + @NotNull + public String getZkEndpoint() + { + return zkEndpoint; + } + + @Config("kafka.zk-endpoint") + public KafkaConnectorConfig setZkEndpoint(String zkEndpoint) + { + this.zkEndpoint = zkEndpoint; + return this; + } + + public int getFetchSize() + { + return fetchSize; + } + + @Config("kafka.fetch-size") + public KafkaConnectorConfig setFetchSize(int fetchSize) + { + this.fetchSize = fetchSize; + return this; + } + public static ImmutableSet parseNodes(String nodes) { Splitter splitter = Splitter.on(',').omitEmptyStrings().trimResults(); diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorFactory.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorFactory.java index cd729032bdb4..88d7410d8c72 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorFactory.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorFactory.java @@ -49,7 +49,7 @@ public class KafkaConnectorFactory @Override public String getName() { - return "kafka"; + return "kafka07"; } @Override diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaInternalFieldDescription.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaInternalFieldDescription.java index 2aa81ed6f9e3..5bcad3d565d4 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaInternalFieldDescription.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaInternalFieldDescription.java @@ -95,12 +95,17 @@ public class KafkaInternalFieldDescription */ public static final KafkaInternalFieldDescription KEY_LENGTH_FIELD = new KafkaInternalFieldDescription("_key_length", BigintType.BIGINT, "Total number of key bytes"); + /** + * _timestamp - offset timestamp, used to narrow scan range + */ + public static final KafkaInternalFieldDescription OFFSET_TIMESTAMP_FIELD = new KafkaInternalFieldDescription("_timestamp", BigintType.BIGINT, "Offset Timestamp"); + public static Set getInternalFields() { return ImmutableSet.of(PARTITION_ID_FIELD, PARTITION_OFFSET_FIELD, SEGMENT_START_FIELD, SEGMENT_END_FIELD, SEGMENT_COUNT_FIELD, KEY_FIELD, KEY_CORRUPT_FIELD, KEY_LENGTH_FIELD, - MESSAGE_FIELD, MESSAGE_CORRUPT_FIELD, MESSAGE_LENGTH_FIELD); + MESSAGE_FIELD, MESSAGE_CORRUPT_FIELD, MESSAGE_LENGTH_FIELD, OFFSET_TIMESTAMP_FIELD); } private final String name; diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaMetadata.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaMetadata.java index 1815646cbda1..5c951f0bc9ea 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaMetadata.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaMetadata.java @@ -27,9 +27,13 @@ import com.facebook.presto.spi.SchemaTablePrefix; import com.facebook.presto.spi.TableNotFoundException; import com.facebook.presto.spi.connector.ConnectorMetadata; +import com.facebook.presto.spi.predicate.Domain; +import com.facebook.presto.spi.predicate.Marker; +import com.facebook.presto.spi.predicate.Range; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import io.airlift.log.Logger; import javax.inject.Inject; @@ -51,6 +55,8 @@ public class KafkaMetadata implements ConnectorMetadata { + private static final Logger log = Logger.get(KafkaMetadata.class); + private final String connectorId; private final boolean hideInternalColumns; private final Map tableDescriptions; @@ -194,7 +200,32 @@ public ColumnMetadata getColumnMetadata(ConnectorSession session, ConnectorTable public List getTableLayouts(ConnectorSession session, ConnectorTableHandle table, Constraint constraint, Optional> desiredColumns) { KafkaTableHandle handle = convertTableHandle(table); - ConnectorTableLayout layout = new ConnectorTableLayout(new KafkaTableLayoutHandle(handle)); + + Long startTs = null; + Long endTs = null; + Optional> domains = constraint.getSummary().getDomains(); + log.info("TableLayout"); + if (domains.isPresent()) { + Map columnHandleDomainMap = domains.get(); + for (Map.Entry entry : columnHandleDomainMap.entrySet()) { + if (entry.getKey() instanceof KafkaColumnHandle && ((KafkaColumnHandle) entry.getKey()).getName().equals(KafkaInternalFieldDescription.OFFSET_TIMESTAMP_FIELD.getName())) { + Range span = entry.getValue().getValues().getRanges().getSpan(); + Marker low = span.getLow(); + Marker high = span.getHigh(); + if (!low.isLowerUnbounded()) { + startTs = (Long) low.getValue(); + } + if (!high.isUpperUnbounded()) { + endTs = (Long) high.getValue(); + } + } + + log.info("K: %s\tV: %s", entry.getKey().toString(), entry.getValue().toString()); + } + } + + log.info("startTs: %s, endTs: %s", startTs, endTs); + ConnectorTableLayout layout = new ConnectorTableLayout(new KafkaTableLayoutHandle(handle, startTs, endTs)); return ImmutableList.of(new ConnectorTableLayoutResult(layout, constraint.getSummary())); } diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java index cb24bd1e031e..4b37298b1a5c 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java @@ -20,6 +20,7 @@ import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.RecordCursor; import com.facebook.presto.spi.RecordSet; +import com.facebook.presto.spi.block.Block; import com.facebook.presto.spi.type.Type; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -27,9 +28,11 @@ import io.airlift.slice.Slice; import io.airlift.slice.Slices; import kafka.api.FetchRequest; -import kafka.api.FetchRequestBuilder; -import kafka.javaapi.FetchResponse; +import kafka.api.OffsetRequest; +import kafka.common.ErrorMapping; +import kafka.common.OffsetOutOfRangeException; import kafka.javaapi.consumer.SimpleConsumer; +import kafka.javaapi.message.ByteBufferMessageSet; import kafka.message.MessageAndOffset; import java.nio.ByteBuffer; @@ -52,11 +55,11 @@ public class KafkaRecordSet { private static final Logger log = Logger.get(KafkaRecordSet.class); - private static final int KAFKA_READ_BUFFER_SIZE = 100_000; private static final byte [] EMPTY_BYTE_ARRAY = new byte [0]; private final KafkaSplit split; private final KafkaSimpleConsumerManager consumerManager; + private final int fetchSize; private final RowDecoder keyDecoder; private final RowDecoder messageDecoder; @@ -74,7 +77,8 @@ public class KafkaRecordSet RowDecoder keyDecoder, RowDecoder messageDecoder, Map> keyFieldDecoders, - Map> messageFieldDecoders) + Map> messageFieldDecoders, + int fetchSize) { this.split = requireNonNull(split, "split is null"); @@ -99,6 +103,7 @@ public class KafkaRecordSet } this.columnTypes = typeBuilder.build(); + this.fetchSize = fetchSize; } @Override @@ -110,7 +115,7 @@ public List getColumnTypes() @Override public RecordCursor cursor() { - return new KafkaRecordCursor(); + return new KafkaRecordCursor(split.getStartTs(), split.getEndTs()); } public class KafkaRecordCursor @@ -120,12 +125,18 @@ public class KafkaRecordCursor private long totalMessages; private long cursorOffset = split.getStart(); private Iterator messageAndOffsetIterator; + private long fetchedSize = 0L; private final AtomicBoolean reported = new AtomicBoolean(); + private final long startTs; + private final long endTs; + private FieldValueProvider[] fieldValueProviders; - KafkaRecordCursor() + KafkaRecordCursor(long startTs, long endTs) { + this.startTs = startTs; + this.endTs = endTs; } @Override @@ -158,32 +169,33 @@ public boolean advanceNextPosition() { while (true) { if (cursorOffset >= split.getEnd()) { - return endOfData(); // Split end is exclusive. + return endOfData(1); // Split end is exclusive. } - // Create a fetch request - openFetchRequest(); - - while (messageAndOffsetIterator.hasNext()) { - MessageAndOffset currentMessageAndOffset = messageAndOffsetIterator.next(); - long messageOffset = currentMessageAndOffset.offset(); - if (messageOffset >= split.getEnd()) { - return endOfData(); // Past our split end. Bail. + try { + // Create a fetch request + openFetchRequest(); + if (cursorOffset >= split.getEnd()) { + return endOfData(2); // Split end is exclusive. } - - if (messageOffset >= cursorOffset) { + while (messageAndOffsetIterator.hasNext()) { + MessageAndOffset currentMessageAndOffset = messageAndOffsetIterator.next(); return nextRow(currentMessageAndOffset); } } + catch (OffsetOutOfRangeException e) { + e.printStackTrace(); + return endOfData(4); + } messageAndOffsetIterator = null; } } - private boolean endOfData() + private boolean endOfData(int from) { if (!reported.getAndSet(true)) { - log.debug("Found a total of %d messages with %d bytes (%d messages expected). Last Offset: %d (%d, %d)", - totalMessages, totalBytes, split.getEnd() - split.getStart(), + log.info("Found (from %d) a total of %d messages with %d bytes (%d compressed bytes expected). Last Offset: %d (%d, %d)", + from, totalMessages, totalBytes, split.getEnd() - split.getStart(), cursorOffset, split.getStart(), split.getEnd()); } return false; @@ -191,17 +203,11 @@ private boolean endOfData() private boolean nextRow(MessageAndOffset messageAndOffset) { - cursorOffset = messageAndOffset.offset() + 1; // Cursor now points to the next message. totalBytes += messageAndOffset.message().payloadSize(); totalMessages++; byte[] keyData = EMPTY_BYTE_ARRAY; byte[] messageData = EMPTY_BYTE_ARRAY; - ByteBuffer key = messageAndOffset.message().key(); - if (key != null) { - keyData = new byte[key.remaining()]; - key.get(keyData); - } ByteBuffer message = messageAndOffset.message().payload(); if (message != null) { @@ -220,6 +226,7 @@ private boolean nextRow(MessageAndOffset messageAndOffset) fieldValueProviders.add(KafkaInternalFieldDescription.KEY_LENGTH_FIELD.forLongValue(keyData.length)); fieldValueProviders.add(KafkaInternalFieldDescription.KEY_CORRUPT_FIELD.forBooleanValue(keyDecoder.decodeRow(keyData, null, fieldValueProviders, columnHandles, keyFieldDecoders))); fieldValueProviders.add(KafkaInternalFieldDescription.MESSAGE_CORRUPT_FIELD.forBooleanValue(messageDecoder.decodeRow(messageData, null, fieldValueProviders, columnHandles, messageFieldDecoders))); + fieldValueProviders.add(KafkaInternalFieldDescription.OFFSET_TIMESTAMP_FIELD.forLongValue(populateOffsetTimestamp(startTs, endTs))); this.fieldValueProviders = new FieldValueProvider[columnHandles.size()]; @@ -239,6 +246,43 @@ private boolean nextRow(MessageAndOffset messageAndOffset) return true; // Advanced successfully. } + private void openFetchRequest() + { + if (messageAndOffsetIterator == null) { + log.info("Fetching %d bytes from partition %d @offset %d (%d - %d) -- %d messages read so far", + fetchSize, split.getPartitionId(), cursorOffset, split.getStart(), split.getEnd(), totalMessages); + cursorOffset += fetchedSize; + FetchRequest req = new FetchRequest(split.getTopicName(), split.getPartitionId(), cursorOffset, fetchSize); + // TODO - this should look at the actual node this is running on and prefer + // that copy if running locally. - look into NodeInfo + SimpleConsumer consumer = consumerManager.getConsumer(split.getLeader()); + + ByteBufferMessageSet fetch = consumer.fetch(req); + log.debug("\t...fetched %s bytes, validBytes=%s, initialOffset=%s", fetch.sizeInBytes(), fetch.validBytes(), fetch.getInitialOffset()); + int errorCode = fetch.getErrorCode(); + if (errorCode != ErrorMapping.NoError() && errorCode != ErrorMapping.OffsetOutOfRangeCode()) { + log.warn("Fetch response has error: %d", errorCode); + throw new PrestoException(KAFKA_SPLIT_ERROR, "could not fetch data from Kafka, error code is '" + errorCode + "'"); + } + + fetchedSize = fetch.validBytes(); + messageAndOffsetIterator = fetch.iterator(); + } + } + + private long populateOffsetTimestamp(long startTs, long endTs) + { + if (startTs == OffsetRequest.EarliestTime()) { + startTs = 0; + } + + if (endTs == OffsetRequest.LatestTime()) { + endTs = Long.MAX_VALUE; + } + + return startTs + (endTs - startTs) / 2; + } + @SuppressWarnings("SimplifiableConditionalExpression") @Override public boolean getBoolean(int field) @@ -279,7 +323,11 @@ public Slice getSlice(int field) @Override public Object getObject(int field) { - throw new UnsupportedOperationException(); + checkArgument(field < columnHandles.size(), "Invalid field index"); + + checkFieldType(field, Block.class); + + return isNull(field) ? null : fieldValueProviders[field].getBlock(); } @Override @@ -300,29 +348,5 @@ private void checkFieldType(int field, Class expected) public void close() { } - - private void openFetchRequest() - { - if (messageAndOffsetIterator == null) { - log.debug("Fetching %d bytes from offset %d (%d - %d). %d messages read so far", KAFKA_READ_BUFFER_SIZE, cursorOffset, split.getStart(), split.getEnd(), totalMessages); - FetchRequest req = new FetchRequestBuilder() - .clientId("presto-worker-" + Thread.currentThread().getName()) - .addFetch(split.getTopicName(), split.getPartitionId(), cursorOffset, KAFKA_READ_BUFFER_SIZE) - .build(); - - // TODO - this should look at the actual node this is running on and prefer - // that copy if running locally. - look into NodeInfo - SimpleConsumer consumer = consumerManager.getConsumer(split.getLeader()); - - FetchResponse fetchResponse = consumer.fetch(req); - if (fetchResponse.hasError()) { - short errorCode = fetchResponse.errorCode(split.getTopicName(), split.getPartitionId()); - log.warn("Fetch response has error: %d", errorCode); - throw new PrestoException(KAFKA_SPLIT_ERROR, "could not fetch data from Kafka, error code is '" + errorCode + "'"); - } - - messageAndOffsetIterator = fetchResponse.messageSet(split.getTopicName(), split.getPartitionId()).iterator(); - } - } } } diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSetProvider.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSetProvider.java index 451cab049d46..ba7b2b9e11a6 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSetProvider.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSetProvider.java @@ -42,12 +42,14 @@ public class KafkaRecordSetProvider { private final KafkaSimpleConsumerManager consumerManager; private final DecoderRegistry registry; + private final KafkaConnectorConfig config; @Inject - public KafkaRecordSetProvider(DecoderRegistry registry, KafkaSimpleConsumerManager consumerManager) + public KafkaRecordSetProvider(DecoderRegistry registry, KafkaSimpleConsumerManager consumerManager, KafkaConnectorConfig config) { this.registry = requireNonNull(registry, "registry is null"); this.consumerManager = requireNonNull(consumerManager, "consumerManager is null"); + this.config = requireNonNull(config, "config is null"); } @Override @@ -90,6 +92,6 @@ public RecordSet getRecordSet(ConnectorTransactionHandle transaction, ConnectorS ImmutableMap> keyFieldDecoders = keyFieldDecoderBuilder.build(); ImmutableMap> messageFieldDecoders = messageFieldDecoderBuilder.build(); - return new KafkaRecordSet(kafkaSplit, consumerManager, handles, keyDecoder, messageDecoder, keyFieldDecoders, messageFieldDecoders); + return new KafkaRecordSet(kafkaSplit, consumerManager, handles, keyDecoder, messageDecoder, keyFieldDecoders, messageFieldDecoders, config.getFetchSize()); } } diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSimpleConsumerManager.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSimpleConsumerManager.java index 5c4ebb023d47..53756c614efa 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSimpleConsumerManager.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSimpleConsumerManager.java @@ -29,7 +29,6 @@ import java.util.concurrent.ExecutionException; import static java.lang.Math.toIntExact; -import static java.lang.String.format; import static java.util.Objects.requireNonNull; /** @@ -94,12 +93,11 @@ private class SimpleConsumerCacheLoader public SimpleConsumer load(HostAddress host) throws Exception { - log.info("Creating new Consumer for %s", host); + log.debug("Creating new Consumer for %s", host); return new SimpleConsumer(host.getHostText(), host.getPort(), connectTimeoutMillis, - bufferSizeBytes, - format("presto-kafka-%s-%s", connectorId, nodeManager.getCurrentNode().getNodeIdentifier())); + bufferSizeBytes); } } } diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplit.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplit.java index ac61361a4287..17edf7d47e6e 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplit.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplit.java @@ -42,6 +42,8 @@ public class KafkaSplit private final long start; private final long end; private final HostAddress leader; + private final long startTs; + private final long endTs; @JsonCreator public KafkaSplit( @@ -52,7 +54,9 @@ public KafkaSplit( @JsonProperty("partitionId") int partitionId, @JsonProperty("start") long start, @JsonProperty("end") long end, - @JsonProperty("leader") HostAddress leader) + @JsonProperty("leader") HostAddress leader, + @JsonProperty("startTs") long startTs, + @JsonProperty("endTs") long endTs) { this.connectorId = requireNonNull(connectorId, "connector id is null"); this.topicName = requireNonNull(topicName, "topicName is null"); @@ -62,6 +66,8 @@ public KafkaSplit( this.start = start; this.end = end; this.leader = requireNonNull(leader, "leader address is null"); + this.startTs = startTs; + this.endTs = endTs; } @JsonProperty @@ -112,6 +118,18 @@ public HostAddress getLeader() return leader; } + @JsonProperty + public long getStartTs() + { + return startTs; + } + + @JsonProperty + public long getEndTs() + { + return endTs; + } + @Override public boolean isRemotelyAccessible() { diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java index f7b436a97b14..2d070d8c29a8 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java @@ -19,31 +19,23 @@ import com.facebook.presto.spi.ConnectorTableLayoutHandle; import com.facebook.presto.spi.FixedSplitSource; import com.facebook.presto.spi.HostAddress; -import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.connector.ConnectorSplitManager; import com.facebook.presto.spi.connector.ConnectorTransactionHandle; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; import io.airlift.log.Logger; -import kafka.api.PartitionOffsetRequestInfo; +import kafka.api.OffsetRequest; import kafka.cluster.Broker; -import kafka.common.TopicAndPartition; -import kafka.javaapi.OffsetRequest; -import kafka.javaapi.OffsetResponse; -import kafka.javaapi.PartitionMetadata; -import kafka.javaapi.TopicMetadata; -import kafka.javaapi.TopicMetadataRequest; -import kafka.javaapi.TopicMetadataResponse; +import kafka.cluster.Cluster; +import kafka.cluster.Partition; import kafka.javaapi.consumer.SimpleConsumer; +import kafka.utils.ZkUtils; +import org.I0Itec.zkclient.ZkClient; import javax.inject.Inject; import java.util.List; -import java.util.Set; import java.util.concurrent.ThreadLocalRandom; -import static com.facebook.presto.kafka.KafkaErrorCode.KAFKA_SPLIT_ERROR; import static com.facebook.presto.kafka.KafkaHandleResolver.convertLayout; import static java.util.Objects.requireNonNull; @@ -57,7 +49,7 @@ public class KafkaSplitManager private final String connectorId; private final KafkaSimpleConsumerManager consumerManager; - private final Set nodes; + private final KafkaConnectorConfig config; @Inject public KafkaSplitManager( @@ -69,76 +61,97 @@ public KafkaSplitManager( this.consumerManager = requireNonNull(consumerManager, "consumerManager is null"); requireNonNull(kafkaConnectorConfig, "kafkaConfig is null"); - this.nodes = ImmutableSet.copyOf(kafkaConnectorConfig.getNodes()); + this.config = kafkaConnectorConfig; } @Override public ConnectorSplitSource getSplits(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableLayoutHandle layout) { KafkaTableHandle kafkaTableHandle = convertLayout(layout).getTable(); + ZkClient zkClient = KafkaUtil.newZkClient(config.getZkEndpoint()); - SimpleConsumer simpleConsumer = consumerManager.getConsumer(selectRandom(nodes)); + ImmutableList.Builder splits = ImmutableList.builder(); + Cluster cluster = ZkUtils.getCluster(zkClient); + List partitions = KafkaUtil.getPartitionsForTopic(zkClient, kafkaTableHandle.getTopicName()); - TopicMetadataRequest topicMetadataRequest = new TopicMetadataRequest(ImmutableList.of(kafkaTableHandle.getTopicName())); - TopicMetadataResponse topicMetadataResponse = simpleConsumer.send(topicMetadataRequest); + long estimatedTotalSize = 0L; - ImmutableList.Builder splits = ImmutableList.builder(); + for (Partition part : partitions) { + log.debug("Adding Partition %s/%s from broker %s", kafkaTableHandle.getTopicName(), part.partId(), part.brokerId()); + Broker leader = cluster.getBroker(part.brokerId()).get(); + + if (leader == null) { // Leader election going on... + log.error("No leader for partition %s/%s found!", kafkaTableHandle.getTopicName(), part.partId()); + continue; + } - for (TopicMetadata metadata : topicMetadataResponse.topicsMetadata()) { - for (PartitionMetadata part : metadata.partitionsMetadata()) { - log.debug("Adding Partition %s/%s", metadata.topic(), part.partitionId()); - - Broker leader = part.leader(); - if (leader == null) { // Leader election going on... - log.warn("No leader for partition %s/%s found!", metadata.topic(), part.partitionId()); - continue; - } - - HostAddress partitionLeader = HostAddress.fromParts(leader.host(), leader.port()); - - SimpleConsumer leaderConsumer = consumerManager.getConsumer(partitionLeader); - // Kafka contains a reverse list of "end - start" pairs for the splits - - long[] offsets = findAllOffsets(leaderConsumer, metadata.topic(), part.partitionId()); - - for (int i = offsets.length - 1; i > 0; i--) { - KafkaSplit split = new KafkaSplit( - connectorId, - metadata.topic(), - kafkaTableHandle.getKeyDataFormat(), - kafkaTableHandle.getMessageDataFormat(), - part.partitionId(), - offsets[i], - offsets[i - 1], - partitionLeader); - splits.add(split); - } + HostAddress partitionLeader = HostAddress.fromParts(leader.host(), leader.port()); + + SimpleConsumer leaderConsumer = consumerManager.getConsumer(partitionLeader); + // Kafka contains a reverse list of "end - start" pairs for the splits + + KafkaTableLayoutHandle layoutHandle = (KafkaTableLayoutHandle) layout; + Long startTs = layoutHandle.getOffsetStartTs(); + Long endTs = layoutHandle.getOffsetEndTs(); + + long[] offsets = findAllOffsets(leaderConsumer, kafkaTableHandle.getTopicName(), part.partId(), startTs, endTs); + for (int i = offsets.length - 1; i > 0; i--) { + KafkaSplit split = new KafkaSplit( + connectorId, + kafkaTableHandle.getTopicName(), + kafkaTableHandle.getKeyDataFormat(), + kafkaTableHandle.getMessageDataFormat(), + part.partId(), + offsets[i], + offsets[i - 1], + partitionLeader, + startTs, + endTs); + splits.add(split); + + long splitSize = (split.getEnd() - split.getStart()) / 1024 / 1024; + log.info("Split summarize: %s-%s (%sMB)", split.getStart(), split.getEnd(), splitSize); + estimatedTotalSize += splitSize; } } - return new FixedSplitSource(splits.build()); + ImmutableList builtSplits = splits.build(); + log.info("Built " + builtSplits.size() + " splits"); + + log.info("EstimatedTotalSize: %s", estimatedTotalSize); + return new FixedSplitSource(builtSplits); } - private static long[] findAllOffsets(SimpleConsumer consumer, String topicName, int partitionId) + private static long[] findAllOffsets(SimpleConsumer consumer, String topicName, int partitionId, Long startTs, Long endTs) { - TopicAndPartition topicAndPartition = new TopicAndPartition(topicName, partitionId); - - // The API implies that this will always return all of the offsets. So it seems a partition can not have - // more than Integer.MAX_VALUE-1 segments. - // - // This also assumes that the lowest value returned will be the first segment available. So if segments have been dropped off, this value - // should not be 0. - PartitionOffsetRequestInfo partitionOffsetRequestInfo = new PartitionOffsetRequestInfo(kafka.api.OffsetRequest.LatestTime(), Integer.MAX_VALUE); - OffsetRequest offsetRequest = new OffsetRequest(ImmutableMap.of(topicAndPartition, partitionOffsetRequestInfo), kafka.api.OffsetRequest.CurrentVersion(), consumer.clientId()); - OffsetResponse offsetResponse = consumer.getOffsetsBefore(offsetRequest); - - if (offsetResponse.hasError()) { - short errorCode = offsetResponse.errorCode(topicName, partitionId); - log.warn("Offset response has error: %d", errorCode); - throw new PrestoException(KAFKA_SPLIT_ERROR, "could not fetch data from Kafka, error code is '" + errorCode + "'"); + // startTs: start timestamp, or -2/null as earliest + // endTs: end timestamp, or -1/null as latest + startTs = startTs == null ? OffsetRequest.EarliestTime() : startTs; + endTs = endTs == null ? OffsetRequest.LatestTime() : endTs; + if (startTs >= endTs && endTs != OffsetRequest.LatestTime()) { + throw new IllegalArgumentException(String.format("Invalid Kafka Offset start/end pair: %s - %s", startTs, endTs)); + } + + long[] offsetsBeforeStartTs = consumer.getOffsetsBefore(topicName, partitionId, startTs, Integer.MAX_VALUE); + long[] offsetsBeforeEndTs = consumer.getOffsetsBefore(topicName, partitionId, endTs, Integer.MAX_VALUE); + log.debug("NumOffsetsBeforeStartTs=%s, NumOffsetsBeforeEndTs=%s", offsetsBeforeStartTs.length, offsetsBeforeEndTs.length); + + if (offsetsBeforeStartTs.length == 0) { + return offsetsBeforeEndTs; + } + + long[] offsets = new long[offsetsBeforeEndTs.length - offsetsBeforeStartTs.length + 1]; + long startOffset = offsetsBeforeStartTs[0]; + + for (int i = 0; i < offsetsBeforeEndTs.length; i++) { + if (offsetsBeforeEndTs[i] == startOffset) { + offsets[i] = startOffset; + break; + } + offsets[i] = offsetsBeforeEndTs[i]; } - return offsetResponse.offsets(topicName, partitionId); + return offsets; } private static T selectRandom(Iterable iterable) diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableLayoutHandle.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableLayoutHandle.java index de0c37816b23..ebe30359143a 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableLayoutHandle.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableLayoutHandle.java @@ -16,6 +16,7 @@ import com.facebook.presto.spi.ConnectorTableLayoutHandle; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import kafka.api.OffsetRequest; import static java.util.Objects.requireNonNull; @@ -23,11 +24,18 @@ public class KafkaTableLayoutHandle implements ConnectorTableLayoutHandle { private final KafkaTableHandle table; + private final Long offsetStartTs; + private final Long offsetEndTs; @JsonCreator - public KafkaTableLayoutHandle(@JsonProperty("table") KafkaTableHandle table) + public KafkaTableLayoutHandle( + @JsonProperty("table") KafkaTableHandle table, + @JsonProperty("offset_start_ts") Long offsetStartTs, + @JsonProperty("offset_end_ts") Long offsetEndTs) { this.table = requireNonNull(table, "table is null"); + this.offsetStartTs = offsetStartTs == null ? OffsetRequest.EarliestTime() : offsetStartTs; + this.offsetEndTs = offsetEndTs == null ? OffsetRequest.LatestTime() : offsetEndTs; } @JsonProperty @@ -36,6 +44,18 @@ public KafkaTableHandle getTable() return table; } + @JsonProperty + public Long getOffsetStartTs() + { + return offsetStartTs; + } + + @JsonProperty + public Long getOffsetEndTs() + { + return offsetEndTs; + } + @Override public String toString() { diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaUtil.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaUtil.java new file mode 100644 index 000000000000..4edf4f6da06f --- /dev/null +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaUtil.java @@ -0,0 +1,159 @@ +/* + * 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.kafka; + +import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; +import kafka.cluster.Broker; +import kafka.cluster.Partition; +import kafka.consumer.ConsumerConfig; +import kafka.consumer.SimpleConsumer; +import kafka.utils.ZKConfig; +import kafka.utils.ZkUtils; +import org.I0Itec.zkclient.ZkClient; +import scala.collection.JavaConversions; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * A collection of utility methods for accessing Kafka. + * + * @author Raghu Angadi + */ +public final class KafkaUtil +{ + public static final ConsumerConfig DEFAULT_CONSUMER_CONFIG; + + static { + Properties properties = new Properties(); + properties.setProperty("groupid", "this-should-not-be-used"); + DEFAULT_CONSUMER_CONFIG = new ConsumerConfig(properties); + } + + private KafkaUtil() + { + } + + /** + * create ZkClient with default options. + */ + public static ZkClient newZkClient(String zkConnect) + { + // get defaults from ZkConfig. + ZKConfig config = new ZKConfig(new Properties()); + + return new ZkClient(zkConnect, + config.zkSessionTimeoutMs(), + config.zkConnectionTimeoutMs(), + kafka.utils.ZKStringSerializer$.MODULE$); + } + + /** + * Returns partitions for given topic. An empty list if the topic is not + * found. + */ + public static List getPartitionsForTopic(ZkClient zkClient, + String topic) + { + // handle scala <-> java conversions. + scala.collection.Iterator topics = + JavaConversions.asScalaIterator(Iterators.forArray(topic)); + Map> map = + JavaConversions.mapAsJavaMap(ZkUtils.getPartitionsForTopics(zkClient, topics)); + + // since we are asking for just one topic, map's size is 0 or 1. + if (map.size() > 0) { + List partitions = JavaConversions.seqAsJavaList( + map.values().iterator().next()); + // transform string to Partition object + return Lists.newArrayList( + Lists.transform(partitions, + input -> Partition.parse(input))); + } + + return new ArrayList<>(); + } + + /** + * Returns latest offset before the given timestamp. If there is no offset + * avaliable, returns the earliest available. An offset before the timestamp + * may not be available if the messages are already rotated. + */ + public static long getBeforeOrEarliestOffset(SimpleConsumer consumer, + String topic, + int partId, + long time) + { + long[] offsets = consumer.getOffsetsBefore(topic, partId, time, 1); + if (offsets.length == 0) { + // then the earliest offset + offsets = consumer.getOffsetsBefore(topic, partId, -2, 1); + } + + return (offsets.length > 0) ? offsets[0] : 0; + } + + /** + * Returns the topics on given Kafka Server + */ + public static List getTopics(ZkClient zkClient) + { + String topicPath = ZkUtils.BrokerTopicsPath(); + return zkClient.getChildren(topicPath); + } + + /** + * Returns the brokers currently registered + */ + public static List getBrokersIds(ZkClient zkClient) + { + String brokerPath = ZkUtils.BrokerIdsPath(); + List brokers = zkClient.getChildren(brokerPath); + List brokerIds = new ArrayList(); + for (String s : brokers) { + Long l = Long.parseLong(s); + brokerIds.add(l); + } + + return brokerIds; + } + + /** + * Returns the number of partitions for a given topic and broker. + */ + public static Integer getNumPartitions(ZkClient zkClient, String topic, Long broker) + { + String topicPath = ZkUtils.BrokerTopicsPath(); + String partitionPath = topicPath + "/" + topic + "/" + broker.toString(); + + String numPartitions = zkClient.readData(partitionPath, true); + if (numPartitions == null) { + return 0; + } + else { + return Integer.parseInt(numPartitions); + } + } + + public static SimpleConsumer newSimpleConsumer(Broker broker) + { + return new SimpleConsumer(broker.host(), broker.port(), + DEFAULT_CONSUMER_CONFIG.socketTimeoutMs(), + DEFAULT_CONSUMER_CONFIG.socketBufferSize()); + } +} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaConnectorConfig.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaConnectorConfig.java index 92bc30a9f160..9c2f836c22b3 100644 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaConnectorConfig.java +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaConnectorConfig.java @@ -32,7 +32,10 @@ public void testDefaults() .setDefaultSchema("default") .setTableNames("") .setTableDescriptionDir(new File("etc/kafka/")) - .setHideInternalColumns(true)); + .setHideInternalColumns(true) + .setFetchSize(10485760) + .setZkEndpoint("") + ); } @Test @@ -46,6 +49,8 @@ public void testExplicitPropertyMappings() .put("kafka.connect-timeout", "1h") .put("kafka.buffer-size", "1MB") .put("kafka.hide-internal-columns", "false") + .put("kafka.fetch-size", "10000000") + .put("kafka.zk-endpoint", "localhost:2181") .build(); KafkaConnectorConfig expected = new KafkaConnectorConfig() @@ -55,7 +60,9 @@ public void testExplicitPropertyMappings() .setNodes("localhost:12345, localhost:23456") .setKafkaConnectTimeout("1h") .setKafkaBufferSize("1MB") - .setHideInternalColumns(false); + .setHideInternalColumns(false) + .setFetchSize(10000000) + .setZkEndpoint("localhost:2181"); ConfigAssertions.assertFullMapping(properties, expected); } diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestManySegments.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestManySegments.java index 4dcba4c146ff..56c39553fe76 100644 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestManySegments.java +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestManySegments.java @@ -22,13 +22,14 @@ import com.facebook.presto.tests.StandaloneQueryRunner; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import kafka.producer.KeyedMessage; +import kafka.javaapi.producer.ProducerData; import org.testng.annotations.AfterClass; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; +import java.util.Collections; import java.util.Properties; import java.util.UUID; @@ -67,9 +68,9 @@ public void startKafka() int jMax = 10_000; int iMax = 100_000 / jMax; for (long i = 0; i < iMax; i++) { - ImmutableList.Builder> builder = ImmutableList.builder(); + ImmutableList.Builder> builder = ImmutableList.builder(); for (long j = 0; j < jMax; j++) { - builder.add(new KeyedMessage(topicName, i, ImmutableMap.of("id", Long.toString(i * iMax + j), "value", UUID.randomUUID().toString()))); + builder.add(new ProducerData<>(topicName, i, Collections.singletonList(ImmutableMap.of("id", String.format("%05d", i * jMax + j), "value", UUID.randomUUID().toString())))); } producer.send(builder.build()); } diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestMinimalFunctionality.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestMinimalFunctionality.java index 1c6b3949e9b8..d26e77b4f9dc 100644 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestMinimalFunctionality.java +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestMinimalFunctionality.java @@ -23,14 +23,16 @@ import com.facebook.presto.spi.type.BigintType; import com.facebook.presto.testing.MaterializedResult; import com.facebook.presto.tests.StandaloneQueryRunner; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import kafka.producer.KeyedMessage; +import kafka.javaapi.producer.ProducerData; import org.testng.annotations.AfterClass; import org.testng.annotations.AfterMethod; import org.testng.annotations.BeforeClass; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; +import java.util.Collections; import java.util.Optional; import java.util.Properties; import java.util.UUID; @@ -96,10 +98,12 @@ public void tearDown() private void createMessages(String topicName, int count) { try (CloseableProducer producer = embeddedKafka.createProducer()) { + ImmutableList.Builder> builder = ImmutableList.builder(); for (long i = 0; i < count; i++) { Object message = ImmutableMap.of("id", Long.toString(i), "value", UUID.randomUUID().toString()); - producer.send(new KeyedMessage<>(topicName, i, message)); + builder.add(new ProducerData<>(topicName, i, Collections.singletonList(message))); } + producer.send(builder.build()); } } @@ -131,7 +135,7 @@ public void testTopicHasData() int count = 1000; createMessages(topicName, count); - + Thread.sleep(10_000); result = queryRunner.execute("SELECT count(1) from " + topicName); expected = MaterializedResult.resultBuilder(SESSION, BigintType.BIGINT) diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedKafka.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedKafka.java index f0db0e309618..7ba6a35db13e 100644 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedKafka.java +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedKafka.java @@ -16,13 +16,11 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.io.Files; -import kafka.admin.AdminUtils; import kafka.javaapi.producer.Producer; import kafka.producer.ProducerConfig; import kafka.server.KafkaConfig; import kafka.server.KafkaServerStartable; -import kafka.utils.ZKStringSerializer$; -import org.I0Itec.zkclient.ZkClient; +import scala.Option; import java.io.Closeable; import java.io.File; @@ -33,7 +31,6 @@ import static com.facebook.presto.kafka.util.TestUtils.findUnusedPort; import static com.facebook.presto.kafka.util.TestUtils.toProperties; -import static com.google.common.base.Preconditions.checkState; import static io.airlift.testing.FileUtils.deleteRecursively; import static java.util.Objects.requireNonNull; @@ -70,23 +67,26 @@ public static EmbeddedKafka createEmbeddedKafka(Properties overrideProperties) this.kafkaDataDir = Files.createTempDir(); Map properties = ImmutableMap.builder() - .put("broker.id", "0") - .put("host.name", "localhost") + .put("brokerid", "0") + .put("regionid", "1") + .put("hostname", "localhost") .put("num.partitions", "2") - .put("log.flush.interval.messages", "10000") - .put("log.flush.interval.ms", "1000") - .put("log.retention.minutes", "60") - .put("log.segment.bytes", "1048576") + .put("log.flush.interval", "10000") + .put("log.default.flush.interval.ms", "1000") + .put("log.retention.hours", "1") + .put("log.segment.bytes", "10248576") + .put("log.file.size", "1048576") .put("auto.create.topics.enable", "false") .put("zookeeper.connection.timeout.ms", "1000000") .put("port", Integer.toString(port)) - .put("log.dirs", kafkaDataDir.getAbsolutePath()) + .put("log.dir", kafkaDataDir.getAbsolutePath()) .put("zookeeper.connect", zookeeper.getConnectString()) + .put("zk.connect", zookeeper.getConnectString()) .putAll(Maps.fromProperties(overrideProperties)) .build(); KafkaConfig config = new KafkaConfig(toProperties(properties)); - this.kafka = new KafkaServerStartable(config); + this.kafka = new KafkaServerStartable(config, Option.empty()); } public void start() @@ -116,26 +116,15 @@ public void createTopics(String... topics) public void createTopics(int partitions, int replication, Properties topicProperties, String... topics) { - checkState(started.get() && !stopped.get(), "not started!"); - - ZkClient zkClient = new ZkClient(getZookeeperConnectString(), 30_000, 30_000, ZKStringSerializer$.MODULE$); - try { - for (String topic : topics) { - AdminUtils.createTopic(zkClient, topic, partitions, replication, topicProperties); - } - } - finally { - zkClient.close(); - } + // noop } public CloseableProducer createProducer() { Map properties = ImmutableMap.builder() - .put("metadata.broker.list", getConnectString()) + .put("broker.list", String.format("0:%s", getConnectString())) .put("serializer.class", JsonEncoder.class.getName()) .put("key.serializer.class", NumberEncoder.class.getName()) - .put("partitioner.class", NumberPartitioner.class.getName()) .put("request.required.acks", "1") .build(); diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedZookeeper.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedZookeeper.java index e5a77f863b42..5b39deda93e1 100644 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedZookeeper.java +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedZookeeper.java @@ -53,8 +53,10 @@ public EmbeddedZookeeper(int port) FileTxnSnapLog ftxn = new FileTxnSnapLog(zkDataDir, zkDataDir); zkServer.setTxnLogFactory(ftxn); + zkServer.setMinSessionTimeout(60000); + zkServer.setMaxSessionTimeout(120000); - cnxnFactory = NIOServerCnxnFactory.createFactory(new InetSocketAddress(port), 0); + cnxnFactory = NIOServerCnxnFactory.createFactory(new InetSocketAddress(port), 300); } public void start() diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/JsonEncoder.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/JsonEncoder.java index 292b184daedb..65e54985c894 100644 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/JsonEncoder.java +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/JsonEncoder.java @@ -11,12 +11,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package com.facebook.presto.kafka.util; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; +import kafka.message.Message; import kafka.serializer.Encoder; -import kafka.utils.VerifiableProperties; import java.io.IOException; @@ -25,17 +26,11 @@ public class JsonEncoder { private final ObjectMapper objectMapper = new ObjectMapper(); - @SuppressWarnings("UnusedParameters") - public JsonEncoder(VerifiableProperties properties) - { - // constructor required by Kafka - } - @Override - public byte[] toBytes(Object o) + public Message toMessage(Object o) { try { - return objectMapper.writeValueAsBytes(o); + return new Message(objectMapper.writeValueAsBytes(o)); } catch (IOException e) { throw Throwables.propagate(e); diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/KafkaLoader.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/KafkaLoader.java index 259fdd7aca53..9a7af55851bc 100644 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/KafkaLoader.java +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/KafkaLoader.java @@ -24,10 +24,11 @@ import com.facebook.presto.tests.ResultsSession; import com.google.common.collect.ImmutableMap; import kafka.javaapi.producer.Producer; -import kafka.producer.KeyedMessage; +import kafka.javaapi.producer.ProducerData; import org.joda.time.format.DateTimeFormatter; import org.joda.time.format.ISODateTimeFormat; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; @@ -109,7 +110,7 @@ public void addResults(QueryResults results) } } - producer.send(new KeyedMessage<>(topicName, count.getAndIncrement(), builder.build())); + producer.send(new ProducerData<>(topicName, count.getAndIncrement(), Collections.singletonList(builder.build()))); } } } diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberEncoder.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberEncoder.java index e0708a5c15d1..07b627b4cff9 100644 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberEncoder.java +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberEncoder.java @@ -13,25 +13,19 @@ */ package com.facebook.presto.kafka.util; +import kafka.message.Message; import kafka.serializer.Encoder; -import kafka.utils.VerifiableProperties; import java.nio.ByteBuffer; public class NumberEncoder implements Encoder { - @SuppressWarnings("UnusedParameters") - public NumberEncoder(VerifiableProperties properties) - { - // constructor required by Kafka - } - @Override - public byte[] toBytes(Number value) + public Message toMessage(Number value) { ByteBuffer buf = ByteBuffer.allocate(8); buf.putLong(value == null ? 0L : value.longValue()); - return buf.array(); + return new Message(buf.array()); } } diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberPartitioner.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberPartitioner.java index d8b5aa3f6eba..4ab86c73f57c 100644 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberPartitioner.java +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberPartitioner.java @@ -13,25 +13,19 @@ */ package com.facebook.presto.kafka.util; +import kafka.cluster.Partition; import kafka.producer.Partitioner; -import kafka.utils.VerifiableProperties; +import scala.collection.Seq; import static java.lang.Math.toIntExact; -public class NumberPartitioner - implements Partitioner +public class NumberPartitioner implements Partitioner { - @SuppressWarnings("UnusedParameters") - public NumberPartitioner(VerifiableProperties properties) - { - // constructor required by Kafka - } - @Override - public int partition(Object key, int numPartitions) + public int partition(Object key, Seq partitions) { if (key instanceof Number) { - return toIntExact(((Number) key).longValue() % numPartitions); + return toIntExact(((Number) key).longValue() % partitions.size()); } return 0; } diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/TestUtils.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/TestUtils.java index 75090a7ab0df..6d5dc53405be 100644 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/TestUtils.java +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/TestUtils.java @@ -64,7 +64,9 @@ public static void installKafkaPlugin(EmbeddedKafka embeddedKafka, QueryRunner q "kafka.nodes", embeddedKafka.getConnectString(), "kafka.table-names", Joiner.on(",").join(topicDescriptions.keySet()), "kafka.connect-timeout", "120s", - "kafka.default-schema", "default"); + "kafka.default-schema", "default", + "kafka.zk-endpoint", embeddedKafka.getZookeeperConnectString() + ); queryRunner.createCatalog("kafka", "kafka", kafkaConfig); } From a5f13cebdfba555d6c029725e88e1da8ca7491b8 Mon Sep 17 00:00:00 2001 From: Xiaoyao Qian Date: Mon, 31 Jul 2017 14:12:28 -0700 Subject: [PATCH 213/331] address comments --- .../facebook/presto/kafka/KafkaRecordSet.java | 32 +++++++++---------- .../presto/kafka/KafkaSplitManager.java | 15 ++------- .../presto/kafka/KafkaTableLayoutHandle.java | 8 ++--- 3 files changed, 23 insertions(+), 32 deletions(-) diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java index 4b37298b1a5c..bd34eeeaf509 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java @@ -178,7 +178,7 @@ public boolean advanceNextPosition() if (cursorOffset >= split.getEnd()) { return endOfData(2); // Split end is exclusive. } - while (messageAndOffsetIterator.hasNext()) { + if (messageAndOffsetIterator.hasNext()) { MessageAndOffset currentMessageAndOffset = messageAndOffsetIterator.next(); return nextRow(currentMessageAndOffset); } @@ -252,21 +252,21 @@ private void openFetchRequest() log.info("Fetching %d bytes from partition %d @offset %d (%d - %d) -- %d messages read so far", fetchSize, split.getPartitionId(), cursorOffset, split.getStart(), split.getEnd(), totalMessages); cursorOffset += fetchedSize; - FetchRequest req = new FetchRequest(split.getTopicName(), split.getPartitionId(), cursorOffset, fetchSize); - // TODO - this should look at the actual node this is running on and prefer - // that copy if running locally. - look into NodeInfo - SimpleConsumer consumer = consumerManager.getConsumer(split.getLeader()); - - ByteBufferMessageSet fetch = consumer.fetch(req); - log.debug("\t...fetched %s bytes, validBytes=%s, initialOffset=%s", fetch.sizeInBytes(), fetch.validBytes(), fetch.getInitialOffset()); - int errorCode = fetch.getErrorCode(); - if (errorCode != ErrorMapping.NoError() && errorCode != ErrorMapping.OffsetOutOfRangeCode()) { - log.warn("Fetch response has error: %d", errorCode); - throw new PrestoException(KAFKA_SPLIT_ERROR, "could not fetch data from Kafka, error code is '" + errorCode + "'"); - } + if (cursorOffset < split.getEnd()) { + FetchRequest req = new FetchRequest(split.getTopicName(), split.getPartitionId(), cursorOffset, fetchSize); + SimpleConsumer consumer = consumerManager.getConsumer(split.getLeader()); + + ByteBufferMessageSet fetch = consumer.fetch(req); + log.debug("\t...fetched %s bytes, validBytes=%s, initialOffset=%s", fetch.sizeInBytes(), fetch.validBytes(), fetch.getInitialOffset()); + int errorCode = fetch.getErrorCode(); + if (errorCode != ErrorMapping.NoError() && errorCode != ErrorMapping.OffsetOutOfRangeCode()) { + log.warn("Fetch response has error: %d", errorCode); + throw new PrestoException(KAFKA_SPLIT_ERROR, "could not fetch data from Kafka, error code is '" + errorCode + "'"); + } - fetchedSize = fetch.validBytes(); - messageAndOffsetIterator = fetch.iterator(); + fetchedSize = fetch.validBytes(); + messageAndOffsetIterator = fetch.iterator(); + } } } @@ -277,7 +277,7 @@ private long populateOffsetTimestamp(long startTs, long endTs) } if (endTs == OffsetRequest.LatestTime()) { - endTs = Long.MAX_VALUE; + endTs = System.currentTimeMillis(); } return startTs + (endTs - startTs) / 2; diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java index 2d070d8c29a8..443cd790166b 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java @@ -34,7 +34,6 @@ import javax.inject.Inject; import java.util.List; -import java.util.concurrent.ThreadLocalRandom; import static com.facebook.presto.kafka.KafkaHandleResolver.convertLayout; import static java.util.Objects.requireNonNull; @@ -91,8 +90,8 @@ public ConnectorSplitSource getSplits(ConnectorTransactionHandle transaction, Co // Kafka contains a reverse list of "end - start" pairs for the splits KafkaTableLayoutHandle layoutHandle = (KafkaTableLayoutHandle) layout; - Long startTs = layoutHandle.getOffsetStartTs(); - Long endTs = layoutHandle.getOffsetEndTs(); + long startTs = layoutHandle.getOffsetStartTs(); + long endTs = layoutHandle.getOffsetEndTs(); long[] offsets = findAllOffsets(leaderConsumer, kafkaTableHandle.getTopicName(), part.partId(), startTs, endTs); for (int i = offsets.length - 1; i > 0; i--) { @@ -122,12 +121,10 @@ public ConnectorSplitSource getSplits(ConnectorTransactionHandle transaction, Co return new FixedSplitSource(builtSplits); } - private static long[] findAllOffsets(SimpleConsumer consumer, String topicName, int partitionId, Long startTs, Long endTs) + private static long[] findAllOffsets(SimpleConsumer consumer, String topicName, int partitionId, long startTs, long endTs) { // startTs: start timestamp, or -2/null as earliest // endTs: end timestamp, or -1/null as latest - startTs = startTs == null ? OffsetRequest.EarliestTime() : startTs; - endTs = endTs == null ? OffsetRequest.LatestTime() : endTs; if (startTs >= endTs && endTs != OffsetRequest.LatestTime()) { throw new IllegalArgumentException(String.format("Invalid Kafka Offset start/end pair: %s - %s", startTs, endTs)); } @@ -153,10 +150,4 @@ private static long[] findAllOffsets(SimpleConsumer consumer, String topicName, return offsets; } - - private static T selectRandom(Iterable iterable) - { - List list = ImmutableList.copyOf(iterable); - return list.get(ThreadLocalRandom.current().nextInt(list.size())); - } } diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableLayoutHandle.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableLayoutHandle.java index ebe30359143a..57e6ffc6e615 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableLayoutHandle.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableLayoutHandle.java @@ -24,8 +24,8 @@ public class KafkaTableLayoutHandle implements ConnectorTableLayoutHandle { private final KafkaTableHandle table; - private final Long offsetStartTs; - private final Long offsetEndTs; + private final long offsetStartTs; + private final long offsetEndTs; @JsonCreator public KafkaTableLayoutHandle( @@ -45,13 +45,13 @@ public KafkaTableHandle getTable() } @JsonProperty - public Long getOffsetStartTs() + public long getOffsetStartTs() { return offsetStartTs; } @JsonProperty - public Long getOffsetEndTs() + public long getOffsetEndTs() { return offsetEndTs; } From 5feb582d6d210d13a5e957e1475b099c9a335dbf Mon Sep 17 00:00:00 2001 From: Xiaoyao Qian Date: Wed, 9 Aug 2017 12:01:58 -0700 Subject: [PATCH 214/331] Fix build and style --- presto-kafka07/pom.xml | 2 +- .../src/main/java/com/facebook/presto/kafka/KafkaMetadata.java | 2 +- .../main/java/com/facebook/presto/kafka/KafkaSplitManager.java | 2 +- .../com/facebook/presto/decoder/thrift/TestThriftDecoder.java | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index ef33044e62ad..96d63ca782ac 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.179-tw-0.36 + 0.181-tw-0.37 presto-kafka07 diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaMetadata.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaMetadata.java index 5c951f0bc9ea..e9644a4c1471 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaMetadata.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaMetadata.java @@ -204,7 +204,7 @@ public List getTableLayouts(ConnectorSession session Long startTs = null; Long endTs = null; Optional> domains = constraint.getSummary().getDomains(); - log.info("TableLayout"); + log.info(constraint.getSummary().toString(null)); if (domains.isPresent()) { Map columnHandleDomainMap = domains.get(); for (Map.Entry entry : columnHandleDomainMap.entrySet()) { diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java index 443cd790166b..a3f372160af3 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java @@ -109,7 +109,7 @@ public ConnectorSplitSource getSplits(ConnectorTransactionHandle transaction, Co splits.add(split); long splitSize = (split.getEnd() - split.getStart()) / 1024 / 1024; - log.info("Split summarize: %s-%s (%sMB)", split.getStart(), split.getEnd(), splitSize); + log.debug("Split summarize: %s-%s (%sMB)", split.getStart(), split.getEnd(), splitSize); estimatedTotalSize += splitSize; } } diff --git a/presto-record-decoder/src/test/java/com/facebook/presto/decoder/thrift/TestThriftDecoder.java b/presto-record-decoder/src/test/java/com/facebook/presto/decoder/thrift/TestThriftDecoder.java index c38c8be0333d..02f8a133606e 100644 --- a/presto-record-decoder/src/test/java/com/facebook/presto/decoder/thrift/TestThriftDecoder.java +++ b/presto-record-decoder/src/test/java/com/facebook/presto/decoder/thrift/TestThriftDecoder.java @@ -73,7 +73,7 @@ public void testSimple() ThriftRowDecoder rowDecoder = new ThriftRowDecoder(); // schema - DecoderTestColumnHandle col1 = new DecoderTestColumnHandle("", 1, "user_id", IntegerType.INTEGER,"1", "thrift", null, false, false, false); + DecoderTestColumnHandle col1 = new DecoderTestColumnHandle("", 1, "user_id", IntegerType.INTEGER, "1", "thrift", null, false, false, false); DecoderTestColumnHandle col2 = new DecoderTestColumnHandle("", 2, "username", createVarcharType(100), "2", "thrift", null, false, false, false); DecoderTestColumnHandle col3 = new DecoderTestColumnHandle("", 3, "text", createVarcharType(100), "3", "thrift", null, false, false, false); DecoderTestColumnHandle col4 = new DecoderTestColumnHandle("", 4, "loc.latitude", DoubleType.DOUBLE, "4/1", "thrift", null, false, false, false); From 8725ff9cc7d2f442a247908afc8be77f096927e2 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 10 Aug 2017 15:12:28 -0700 Subject: [PATCH 215/331] Tableau Presto ODBC Driver fix --- .../presto/server/StatementResource.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/presto-main/src/main/java/com/facebook/presto/server/StatementResource.java b/presto-main/src/main/java/com/facebook/presto/server/StatementResource.java index 684a77b642cf..4d6daf04eb3e 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/StatementResource.java +++ b/presto-main/src/main/java/com/facebook/presto/server/StatementResource.java @@ -167,6 +167,23 @@ public Response createQuery( .build()); } + // The Teradata Presto ODBC Driver checks node version to decide the Presto's statement + // protocol and sends test queries about PREPARE statement. + // Rewrite the statement so that Presto always returns version for the compatible protocol. + // Ban the statement which will never be consumed by the driver. + if (servletRequest.getHeader("User-Agent").equals("Teradata Presto ODBC Driver")) { + if (statement.equals("select node_version from system.runtime.nodes where coordinator=true")) { + statement = "select '0.148' as node_version"; + } + else if (statement.equals("DESCRIBE OUTPUT prepare_test_stmt")) { + return Response + .status(Status.BAD_REQUEST) + .type(MediaType.TEXT_PLAIN) + .entity("SQL statement is known, and wouldn't be consumed by this driver") + .build(); + } + } + SessionSupplier sessionSupplier = new HttpRequestSessionFactory(servletRequest); ExchangeClient exchangeClient = exchangeClientSupplier.get(deltaMemoryInBytes -> { }); From b7ae0e21f458e75f8d12ad5206ac1862de920392 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Mon, 14 Aug 2017 16:29:52 -0700 Subject: [PATCH 216/331] Fix test --- .../java/com/facebook/presto/server/StatementResource.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/presto-main/src/main/java/com/facebook/presto/server/StatementResource.java b/presto-main/src/main/java/com/facebook/presto/server/StatementResource.java index 4d6daf04eb3e..5658e51c9ea3 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/StatementResource.java +++ b/presto-main/src/main/java/com/facebook/presto/server/StatementResource.java @@ -104,6 +104,8 @@ import static com.facebook.presto.util.Failures.toFailure; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Strings.isNullOrEmpty; +import static com.google.common.base.Strings.nullToEmpty; + import static io.airlift.concurrent.Threads.threadsNamed; import static io.airlift.http.client.HttpUriBuilder.uriBuilderFrom; import static io.airlift.units.DataSize.Unit.MEGABYTE; @@ -171,7 +173,7 @@ public Response createQuery( // protocol and sends test queries about PREPARE statement. // Rewrite the statement so that Presto always returns version for the compatible protocol. // Ban the statement which will never be consumed by the driver. - if (servletRequest.getHeader("User-Agent").equals("Teradata Presto ODBC Driver")) { + if (nullToEmpty(servletRequest.getHeader("User-Agent")).equals("Teradata Presto ODBC Driver")) { if (statement.equals("select node_version from system.runtime.nodes where coordinator=true")) { statement = "select '0.148' as node_version"; } From e092c37802b62ed6b782c98ba4063cd6fdd32282 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 17 Aug 2017 15:58:58 -0700 Subject: [PATCH 217/331] version 0.181-tw-0.38 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 50 files changed, 52 insertions(+), 52 deletions(-) diff --git a/pom.xml b/pom.xml index 9410185326b7..a947ac623a26 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.181-tw-0.37 + 0.181-tw-0.38 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index c7b0cb709357..3d8d24ef1aa5 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index a34e9edf331c..c8286d6fb056 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 3e312dd41255..e194d5377883 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 4c5e0f90e74b..c955a7af0ba5 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 19b38c1c5ff9..de70af354d24 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 99942fab22ca..d8877d5276ca 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.181-tw-0.37 + 0.181-tw-0.38 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 45cac6b08720..51769a463816 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index d3fa947ee109..06f0e79bc6ce 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 75e1d227acf7..87ce5bcbbf96 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 2a499ac16bbb..1820580521d8 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 59ff3b03bb86..dc865a45b1b4 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 564ef54c8b3d..b4771f3c668e 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 3382fdd23f25..8f6c4776e97a 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 601f42469d35..efec98597888 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-example-http diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index f509c2b7401d..ed13cfb06105 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 4c2783422b4a..3dc1ad4ffa7a 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 3012eb3044c1..43b32aa30c70 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 94ef3bc69c7a..b32f619e68dc 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 38ad83f0bf10..564e085674b7 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index 96d63ca782ac..e95b36d1a4a1 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 483e0b2246da..98e5d63a4e4e 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 357dc08934ff..1d46345d4ced 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-main diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 33a14399fc04..7d08ad39540f 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 9d1f11240be4..586674c244c9 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 29cca6c69338..e6716d76fe03 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 1e188a5e822a..c6ddcc2da192 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 8f5fa1f55d2d..6c1bdd465d4d 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 27d1e819e309..11c61b583caa 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 5ad3f5f8b728..c3d1f81fd241 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index a9ee0664bec0..e8e035c5c98b 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 7d55006a1c6f..94583e544f95 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.181-tw-0.37 + 0.181-tw-0.38 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index bbebb06643fa..6751b4e9925c 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index af113f8ae402..f4bffec6829c 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 11bfc2fab069..7cc9e4244664 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 087b22d1164d..03007d89db29 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 81722baea1d8..fbe2a5b3b634 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 4e1096e3cf57..1cc354ca64e5 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 9df27545885e..99741181bcdc 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 5367e6981c7f..088f4f4a53a6 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index dd44e45bfc93..b84f177d0c68 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.181-tw-0.37 + 0.181-tw-0.38 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 77c6f701c613..429c333bd92f 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 7f5b625a7f1d..a045fde9328a 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index bc41260dfdc3..91cad28e28f5 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.181-tw-0.37 + 0.181-tw-0.38 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index c8b7b537ee95..e32c46b28fe0 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index a7d885bfaea2..15466a1423e0 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index bf87b07f0783..e61fbc85667d 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-thrift-testing-server diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 1f6423a66dd1..bb57d7643721 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index d8db6ca11a3c..e1577901ac52 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 6dbbc9f3f689..66f19f3158f9 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.37 + 0.181-tw-0.38 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.181-tw-0.37 + 0.181-tw-0.38 provided From 2f6c8c7004941fef4205573a467c2d9087569a0d Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Mon, 21 Aug 2017 15:21:10 -0700 Subject: [PATCH 218/331] Fix kafka07 unit test --- .../com/facebook/presto/kafka/TestKafkaConnectorConfig.java | 2 +- .../src/test/java/com/facebook/presto/kafka/util/TestUtils.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaConnectorConfig.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaConnectorConfig.java index 9c2f836c22b3..06bfbf3d6474 100644 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaConnectorConfig.java +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaConnectorConfig.java @@ -31,7 +31,7 @@ public void testDefaults() .setKafkaBufferSize("64kB") .setDefaultSchema("default") .setTableNames("") - .setTableDescriptionDir(new File("etc/kafka/")) + .setTableDescriptionDir(new File("etc/kafka07/")) .setHideInternalColumns(true) .setFetchSize(10485760) .setZkEndpoint("") diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/TestUtils.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/TestUtils.java index 6d5dc53405be..3c9be963114e 100644 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/TestUtils.java +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/TestUtils.java @@ -67,7 +67,7 @@ public static void installKafkaPlugin(EmbeddedKafka embeddedKafka, QueryRunner q "kafka.default-schema", "default", "kafka.zk-endpoint", embeddedKafka.getZookeeperConnectString() ); - queryRunner.createCatalog("kafka", "kafka", kafkaConfig); + queryRunner.createCatalog("kafka", "kafka07", kafkaConfig); } public static void loadTpchTopic(EmbeddedKafka embeddedKafka, TestingPrestoClient prestoClient, String topicName, QualifiedObjectName tpchTableName) From 1501e503d5d80816b6591b822326a6d37306a42b Mon Sep 17 00:00:00 2001 From: thomass Date: Wed, 30 Aug 2017 10:25:26 -0700 Subject: [PATCH 219/331] add cumulative memory and peak memeory --- twitter-eventlistener-plugin/pom.xml | 2 +- .../presto/plugin/eventlistener/QueryCompletedEventScriber.java | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 66f19f3158f9..ce8f5915ceb3 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -45,7 +45,7 @@ com.twitter presto-thrift-java - 0.0.3 + 0.0.4 com.twitter diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java index 120535ad3cfa..dfd5b2208c42 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java @@ -86,6 +86,8 @@ private static QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletedE thriftEvent.end_time_ms = event.getEndTime().toEpochMilli(); thriftEvent.queued_time_ms = eventStat.getQueuedTime().toMillis(); thriftEvent.query_wall_time_ms = eventStat.getWallTime().toMillis(); + thriftEvent.cumulative_memory_bytesecond = eventStat.getCumulativeMemory(); + thriftEvent.peak_memory_bytes = eventStat.getPeakMemoryBytes(); if (eventStat.getAnalysisTime().isPresent()) { thriftEvent.analysis_time_ms = eventStat.getAnalysisTime().get().toMillis(); } From 70023d44619f32621a60e31a872835e4289992eb Mon Sep 17 00:00:00 2001 From: thomass Date: Thu, 31 Aug 2017 14:29:13 -0700 Subject: [PATCH 220/331] prepare release 0.181-tw-0.39 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 50 files changed, 52 insertions(+), 52 deletions(-) diff --git a/pom.xml b/pom.xml index a947ac623a26..831a7372a3da 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.181-tw-0.38 + 0.181-tw-0.39 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 3d8d24ef1aa5..caba67b10469 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index c8286d6fb056..3b453d7ed602 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index e194d5377883..d058d420dfc9 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index c955a7af0ba5..253be3f4f828 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index de70af354d24..84684425eaad 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index d8877d5276ca..c1378e82548a 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.181-tw-0.38 + 0.181-tw-0.39 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 51769a463816..36738d7a5a60 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 06f0e79bc6ce..f981a9b97747 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 87ce5bcbbf96..4915d365c9b5 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 1820580521d8..6945381febf5 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index dc865a45b1b4..e3bc71d4bd61 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index b4771f3c668e..255f0f4da066 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 8f6c4776e97a..ebadda9a1234 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index efec98597888..85c92bbbdf36 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-example-http diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index ed13cfb06105..0a1f9f5786a5 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 3dc1ad4ffa7a..beeb70d05a48 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 43b32aa30c70..ff207598132a 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index b32f619e68dc..b64533843c52 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 564e085674b7..7178c908f307 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index e95b36d1a4a1..d2d51bd1c746 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 98e5d63a4e4e..68364a504dcc 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 1d46345d4ced..690581f23e91 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-main diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 7d08ad39540f..3cc1c4d7aa16 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 586674c244c9..50145f28d1a6 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index e6716d76fe03..f3ce0df6c2a7 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index c6ddcc2da192..181d62aa9ed8 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 6c1bdd465d4d..7829ce3ad967 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 11c61b583caa..314f4138d03d 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index c3d1f81fd241..40928cb0c66c 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index e8e035c5c98b..25bad2300d3d 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 94583e544f95..43474d17c92c 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.181-tw-0.38 + 0.181-tw-0.39 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 6751b4e9925c..d8937574a666 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index f4bffec6829c..ae813f72db8b 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 7cc9e4244664..08a272642d87 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 03007d89db29..8f7fb4e8778a 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index fbe2a5b3b634..d6b082348cf9 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 1cc354ca64e5..7bbd20ee6070 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 99741181bcdc..b18a1b4f11bf 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 088f4f4a53a6..0d487f302f3a 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index b84f177d0c68..bb06e51737ad 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.181-tw-0.38 + 0.181-tw-0.39 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 429c333bd92f..b3f28afde8aa 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index a045fde9328a..48ec4ff2f48a 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 91cad28e28f5..dde9938eaee7 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.181-tw-0.38 + 0.181-tw-0.39 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index e32c46b28fe0..676435bd1448 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 15466a1423e0..8e64b53209f3 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index e61fbc85667d..8b81be71a997 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-thrift-testing-server diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index bb57d7643721..43e6a4a4d26e 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index e1577901ac52..3303ca264fd4 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index ce8f5915ceb3..c63529619ace 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.38 + 0.181-tw-0.39 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.181-tw-0.38 + 0.181-tw-0.39 provided From 13d003224576b79aeeeee2a9d47872cb5c695bb5 Mon Sep 17 00:00:00 2001 From: thomass Date: Mon, 25 Sep 2017 16:25:20 -0700 Subject: [PATCH 221/331] add cpuTime_ms --- .../presto/plugin/eventlistener/QueryCompletedEventScriber.java | 1 + 1 file changed, 1 insertion(+) diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java index dfd5b2208c42..f6e13956b4dc 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java @@ -88,6 +88,7 @@ private static QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletedE thriftEvent.query_wall_time_ms = eventStat.getWallTime().toMillis(); thriftEvent.cumulative_memory_bytesecond = eventStat.getCumulativeMemory(); thriftEvent.peak_memory_bytes = eventStat.getPeakMemoryBytes(); + thriftEvent.cpu_time_ms = eventStat.getCpuTime().toMillis(); if (eventStat.getAnalysisTime().isPresent()) { thriftEvent.analysis_time_ms = eventStat.getAnalysisTime().get().toMillis(); } From b7a4ed1d6b5d41212c358ed1288dffcbd73723bb Mon Sep 17 00:00:00 2001 From: thomass Date: Mon, 25 Sep 2017 16:28:55 -0700 Subject: [PATCH 222/331] update thrift package version --- twitter-eventlistener-plugin/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index c63529619ace..145708cb8d7e 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -45,7 +45,7 @@ com.twitter presto-thrift-java - 0.0.4 + 0.0.5 com.twitter From ad476ae0849b4d767b5511e6114432e98c400e76 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Mon, 2 Oct 2017 14:28:03 -0700 Subject: [PATCH 223/331] Added file path for error splitting blocks --- .../com/facebook/presto/hive/BackgroundHiveSplitLoader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 43f6536e28ac..c80aceb06c68 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 @@ -624,7 +624,7 @@ protected HiveSplit computeNext() while (chunkOffset >= blockLocation.getLength()) { // allow overrun for lzo compressed file for intermediate blocks if (!isLzopCompressedFile(filePath) || blockLocation.getOffset() + blockLocation.getLength() >= length) { - checkState(chunkOffset == blockLocation.getLength(), "Error splitting blocks"); + checkState(chunkOffset == blockLocation.getLength(), "Error splitting blocks for file: " + filePath.toString() + ""); } blockLocationIterator.next(); chunkOffset -= blockLocation.getLength(); From 07ade559a611ecafc80cf9f400bbaead1998645e Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 13 Oct 2017 01:35:39 -0700 Subject: [PATCH 224/331] Fix Travis CI for twitter-master branch. Split twitter-modules into seperate profile Add presto-twitter-server Alter location formulation for the temp folder Switch Travis badge in README Setup twitter notification Remove broken cli and hive connector related tests in presto-product-test --- .travis.yml | 43 +- README.md | 2 +- pom.xml | 13 +- presto-hive-hadoop2/bin/run_on_docker.sh | 2 +- .../facebook/presto/hive/HiveWriteUtils.java | 18 +- presto-server/src/main/provisio/presto.xml | 12 - presto-twitter-server/NOTICE | 2836 +++++++++++++++++ presto-twitter-server/README.txt | 5 + presto-twitter-server/pom.xml | 26 + .../src/main/provisio/presto.xml | 64 + .../src/main/provisio/twitter.xml | 16 + 11 files changed, 3007 insertions(+), 30 deletions(-) create mode 100644 presto-twitter-server/NOTICE create mode 100644 presto-twitter-server/README.txt create mode 100644 presto-twitter-server/pom.xml create mode 100644 presto-twitter-server/src/main/provisio/presto.xml create mode 100644 presto-twitter-server/src/main/provisio/twitter.xml diff --git a/.travis.yml b/.travis.yml index 227521b43994..73a8ea4bafea 100644 --- a/.travis.yml +++ b/.travis.yml @@ -32,27 +32,43 @@ addons: cache: directories: - $HOME/.m2/repository + - $HOME/.thrift services: - docker +before_install: + - | + if [[ ! -e $HOME/.thrift/bin/thrift ]]; then + sudo apt-get install libboost-dev libboost-test-dev libboost-program-options-dev libboost-filesystem-dev libboost-thread-dev libevent-dev automake libtool flex bison pkg-config g++ libssl-dev + wget https://www.apache.org/dist/thrift/0.9.3/thrift-0.9.3.tar.gz + tar xfz thrift-0.9.3.tar.gz + cd thrift-0.9.3 && ./configure --without-cpp --without-c_glib --without-python --without-ruby --without-php --without-erlang --without-go --without-nodejs -q --prefix=$HOME/.thrift + sudo make install > thrift_make_install.log + cd .. + fi + - | + if [[ ! -e /usr/local/bin/thrift ]]; then + sudo ln -s $HOME/.thrift/bin/thrift /usr/local/bin/thrift + fi + install: - ./mvnw -v - | if [[ -v TEST_SPECIFIC_MODULES ]]; then - ./mvnw install $MAVEN_FAST_INSTALL -pl $TEST_SPECIFIC_MODULES -am + ./mvnw install $MAVEN_FAST_INSTALL -P !twitter-modules -pl $TEST_SPECIFIC_MODULES -am fi - | if [[ -v TEST_OTHER_MODULES ]]; then - ./mvnw install $MAVEN_FAST_INSTALL -pl '!presto-docs,!presto-server,!presto-server-rpm' + ./mvnw install $MAVEN_FAST_INSTALL -P !twitter-modules -pl '!presto-docs,!presto-server,!presto-server-rpm' fi - | if [[ -v PRODUCT_TESTS_BASIC_ENVIRONMENT || -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then - ./mvnw install $MAVEN_FAST_INSTALL -pl '!presto-docs,!presto-server-rpm' + ./mvnw install $MAVEN_FAST_INSTALL -P !twitter-modules -pl '!presto-docs,!presto-server-rpm' fi - | if [[ -v HIVE_TESTS ]]; then - ./mvnw install $MAVEN_FAST_INSTALL -pl presto-hive-hadoop2 -am + ./mvnw install $MAVEN_FAST_INSTALL -P !twitter-modules -pl presto-hive-hadoop2 -am fi before_script: @@ -65,30 +81,30 @@ before_script: script: - | if [[ -v MAVEN_CHECKS ]]; then - ./mvnw install -DskipTests -B -T C1 + ./mvnw install -DskipTests -B -T C1 -P !twitter-modules fi - | if [[ -v TEST_SPECIFIC_MODULES ]]; then - ./mvnw test $MAVEN_SKIP_CHECKS_AND_DOCS -B -pl $TEST_SPECIFIC_MODULES $TEST_FLAGS + ./mvnw test $MAVEN_SKIP_CHECKS_AND_DOCS -P !twitter-modules -B -pl $TEST_SPECIFIC_MODULES $TEST_FLAGS fi - | if [[ -v TEST_OTHER_MODULES ]]; then - ./mvnw test $MAVEN_SKIP_CHECKS_AND_DOCS -B -pl $TEST_OTHER_MODULES + ./mvnw test $MAVEN_SKIP_CHECKS_AND_DOCS -P !twitter-modules -B -pl $TEST_OTHER_MODULES fi - | 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 + multinode -x quarantine,big_query,storage_formats,profile_specific_tests,tpcds,cli,hive_connector fi - | 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 + singlenode-kerberos-hdfs-impersonation -g storage_formats,hdfs_impersonation,authorization fi - | if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then presto-product-tests/bin/run_on_docker.sh \ - singlenode-ldap -g ldap -x simba_jdbc + singlenode-ldap -g ldap -x simba_jdbc,ldap_cli fi # SQL server image sporadically hangs during the startup # TODO: Uncomment it once issue is fixed @@ -101,7 +117,7 @@ script: - | if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then presto-product-tests/bin/run_on_docker.sh \ - multinode-tls -g smoke,cli,group-by,join,tls + multinode-tls -g smoke,group-by,join,tls fi - | if [[ -v HIVE_TESTS ]]; then @@ -113,8 +129,9 @@ before_cache: - rm -rf $HOME/.m2/repository/com/facebook notifications: - slack: - secure: V5eyoGShxFoCcYJcp858vf/T6gC9KeMxL0C1EElcpZRcKBrIVZzvhek3HLHxZOxlghqnvNVsyDtU3u5orkEaAXeXj5c2dN+4XBsAB9oeN5MtQ0Z3VLAhZDqKIW1LzcXrq4DpzM0PkGhjfjum/P94/qFYk0UckPtB6a341AuYRo8= + hipchat: + rooms: + secure: peNh1KxwlxIpFyb60S8AMvaJThgh1LsjE+Whf1rYkJalVd2wUrqBIoyDKVSueyHD01hQ06gT7rBV6Pu/QcBMR1a9BbMCjERfxLZFUAheuC2Rsb+p1c4dyvBcFUGacgW7XWKCaVYGDGxuUvb0I3Z8cR6KxhK2xi88tHiqBGVGV2yI6zzOTpWVknMfFBtn+ONU1Ob2P6trclXaDyFd4MxubULri6CQdl35eQAq/VnmR3SZOgyVu3V30MGKwI3zhSli+3VqmW0JmaDGoHN6gznM1+VqABLgmIq0P+n+r5gdZWRCorq10NZCFMhVQ8U6rQHcL7sAniYJJsC/yRt6+pjyzIF4N+LSzZ7T+FLxQqT7k/1ukNgrujLDfTpn76Mo9eYTZmfAdzbm1QKJDACwr8Slqhq1jGzcrFMHunvXhVqjOs24R+JAHblY0O9PXvv7aR29GOQWDCvD7nV5QBUr8Xz5q7ozbLqHTI+yH02Jj4EaZ+azWYdRmnr9wDBxWMYBEgOdj4pII9b298XEDB72TxA3KpLTpdLxBTR+gIk/LjJqb/wb84xUv8gPXkaXccltGd5YI90c84cX8isbzNkAylzyfF2Eyueh0XbnMHfpFqBS7qaVM0/D+UxZkU0WNJ0x7G9XJvkiq49bZz2q1KLE4XuvVnTZSSjVSUAS8RtHfwUV33c= before_deploy: - mkdir /tmp/artifacts diff --git a/README.md b/README.md index f7428dba837d..9325bc3edef5 100644 --- a/README.md +++ b/README.md @@ -1,4 +1,4 @@ -# Presto [![Build Status](https://travis-ci.org/prestodb/presto.svg?branch=master)](https://travis-ci.org/prestodb/presto) +# Presto [![Build Status](https://travis-ci.org/twitter-forks/presto.svg?branch=twitter-master)](https://travis-ci.org/twitter-forks/presto) Presto is a distributed SQL query engine for big data. diff --git a/pom.xml b/pom.xml index 831a7372a3da..43c4c5ed0d5a 100644 --- a/pom.xml +++ b/pom.xml @@ -73,7 +73,6 @@ presto-jmx presto-record-decoder presto-kafka - presto-kafka07 presto-redis presto-accumulo presto-cassandra @@ -85,7 +84,6 @@ presto-hive-hadoop2 presto-teradata-functions presto-example-http - twitter-eventlistener-plugin presto-local-file presto-tpch presto-raptor @@ -1207,5 +1205,16 @@ + + twitter-modules + + true + + + presto-kafka07 + twitter-eventlistener-plugin + presto-twitter-server + + diff --git a/presto-hive-hadoop2/bin/run_on_docker.sh b/presto-hive-hadoop2/bin/run_on_docker.sh index 08533e1a7c84..f0049e280ab5 100755 --- a/presto-hive-hadoop2/bin/run_on_docker.sh +++ b/presto-hive-hadoop2/bin/run_on_docker.sh @@ -108,7 +108,7 @@ fi # run product tests pushd $PROJECT_ROOT set +e -./mvnw -pl presto-hive-hadoop2 test -P test-hive-hadoop2 \ +./mvnw -pl presto-hive-hadoop2 test -P test-hive-hadoop2,!twitter-modules \ -Dhive.hadoop2.timeZone=UTC \ -DHADOOP_USER_NAME=hive \ -Dhive.hadoop2.metastoreHost=hadoop-master \ diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java index 70595498adbb..9a6ab1497ffd 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java @@ -441,6 +441,17 @@ public static boolean isViewFileSystem(String user, HdfsEnvironment hdfsEnvironm } } + public static boolean isHDFSCompatibleViewFileSystem(String user, HdfsEnvironment hdfsEnvironment, Path path) + { + try { + return hdfsEnvironment.getFileSystem(user, path) + .getClass().getName().equals("org.apache.hadoop.fs.viewfs.HDFSCompatibleViewFileSystem"); + } + catch (IOException e) { + throw new PrestoException(HIVE_FILESYSTEM_ERROR, "Failed checking path: " + path, e); + } + } + public static boolean isLocalFileSystem(String user, HdfsEnvironment hdfsEnvironment, Path path) { try { @@ -465,7 +476,7 @@ private static boolean isDirectory(String user, HdfsEnvironment hdfsEnvironment, public static Path createTemporaryPath(String user, HdfsEnvironment hdfsEnvironment, Path targetPath) { // use a per-user temporary directory to avoid permission problems - String temporaryPrefix = "/user/" + user + "/warehouse/.hive-staging"; + String temporaryPrefix = "/tmp/presto-" + user; // use a per-user temporary directory in local fs system if (isLocalFileSystem(user, hdfsEnvironment, targetPath)) { @@ -477,6 +488,11 @@ public static Path createTemporaryPath(String user, HdfsEnvironment hdfsEnvironm temporaryPrefix = ".hive-staging"; } + // use relative temporary directory on HDFSCompatibleViewFileSystem + if (isHDFSCompatibleViewFileSystem(user, hdfsEnvironment, targetPath)) { + temporaryPrefix = "../.hive-staging"; + } + // create a temporary directory on the same filesystem Path temporaryRoot = new Path(targetPath, temporaryPrefix); Path temporaryPath = new Path(temporaryRoot, randomUUID().toString()); diff --git a/presto-server/src/main/provisio/presto.xml b/presto-server/src/main/provisio/presto.xml index 7ff4a8b506bc..6e11d70860aa 100644 --- a/presto-server/src/main/provisio/presto.xml +++ b/presto-server/src/main/provisio/presto.xml @@ -74,12 +74,6 @@ - - - - - - @@ -152,12 +146,6 @@ - - - - - - diff --git a/presto-twitter-server/NOTICE b/presto-twitter-server/NOTICE new file mode 100644 index 000000000000..6182bbb1148d --- /dev/null +++ b/presto-twitter-server/NOTICE @@ -0,0 +1,2836 @@ +THE FOLLOWING IS SOFTWARE LICENSED BY THIRD PARTIES UNDER OPEN SOURCE LICENSES THAT MAY BE USED BY THIS PRODUCT. + +----- + +The following software may be included in this product: aether. The source code is available at http://eclipse.org/aether/download/. You may also request a copy of the source code by sending a request to opensource@fb.com. This software contains the following license and notice below: + +Eclipse Public License - v 1.0 + +THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. + +1. DEFINITIONS + +"Contribution" means: + +a) in the case of the initial Contributor, the initial code and documentation distributed under this Agreement, and + +b) in the case of each subsequent Contributor: + +i) changes to the Program, and + +ii) additions to the Program; + +where such changes and/or additions to the Program originate from and are distributed by that particular Contributor. A Contribution 'originates' from a Contributor if it was added to the Program by such Contributor itself or anyone acting on such Contributor's behalf. Contributions do not include additions to the Program which: (i) are separate modules of software distributed in conjunction with the Program under their own license agreement, and (ii) are not derivative works of the Program. + +"Contributor" means any person or entity that distributes the Program. + +"Licensed Patents" mean patent claims licensable by a Contributor which are necessarily infringed by the use or sale of its Contribution alone or when combined with the Program. + +"Program" means the Contributions distributed in accordance with this Agreement. + +"Recipient" means anyone who receives the Program under this Agreement, including all Contributors. + +2. GRANT OF RIGHTS + +a) Subject to the terms of this Agreement, each Contributor hereby grants Recipient a non-exclusive, worldwide, royalty-free copyright license to reproduce, prepare derivative works of, publicly display, publicly perform, distribute and sublicense the Contribution of such Contributor, if any, and such derivative works, in source code and object code form. + +b) Subject to the terms of this Agreement, each Contributor hereby grants Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed Patents to make, use, sell, offer to sell, import and otherwise transfer the Contribution of such Contributor, if any, in source code and object code form. This patent license shall apply to the combination of the Contribution and the Program if, at the time the Contribution is added by the Contributor, such addition of the Contribution causes such combination to be covered by the Licensed Patents. The patent license shall not apply to any other combinations which include the Contribution. No hardware per se is licensed hereunder. + +c) Recipient understands that although each Contributor grants the licenses to its Contributions set forth herein, no assurances are provided by any Contributor that the Program does not infringe the patent or other intellectual property rights of any other entity. Each Contributor disclaims any liability to Recipient for claims brought by any other entity based on infringement of intellectual property rights or otherwise. As a condition to exercising the rights and licenses granted hereunder, each Recipient hereby assumes sole responsibility to secure any other intellectual property rights needed, if any. For example, if a third party patent license is required to allow Recipient to distribute the Program, it is Recipient's responsibility to acquire that license before distributing the Program. + +d) Each Contributor represents that to its knowledge it has sufficient copyright rights in its Contribution, if any, to grant the copyright license set forth in this Agreement. + +3. REQUIREMENTS + +A Contributor may choose to distribute the Program in object code form under its own license agreement, provided that: + +a) it complies with the terms and conditions of this Agreement; and + +b) its license agreement: + +i) effectively disclaims on behalf of all Contributors all warranties and conditions, express and implied, including warranties or conditions of title and non-infringement, and implied warranties or conditions of merchantability and fitness for a particular purpose; + +ii) effectively excludes on behalf of all Contributors all liability for damages, including direct, indirect, special, incidental and consequential damages, such as lost profits; + +iii) states that any provisions which differ from this Agreement are offered by that Contributor alone and not by any other party; and + +iv) states that source code for the Program is available from such Contributor, and informs licensees how to obtain it in a reasonable manner on or through a medium customarily used for software exchange. + +When the Program is made available in source code form: + +a) it must be made available under this Agreement; and + +b) a copy of this Agreement must be included with each copy of the Program. + +Contributors may not remove or alter any copyright notices contained within the Program. + +Each Contributor must identify itself as the originator of its Contribution, if any, in a manner that reasonably allows subsequent Recipients to identify the originator of the Contribution. + +4. COMMERCIAL DISTRIBUTION + +Commercial distributors of software may accept certain responsibilities with respect to end users, business partners and the like. While this license is intended to facilitate the commercial use of the Program, the Contributor who includes the Program in a commercial product offering should do so in a manner which does not create potential liability for other Contributors. Therefore, if a Contributor includes the Program in a commercial product offering, such Contributor ("Commercial Contributor") hereby agrees to defend and indemnify every other Contributor ("Indemnified Contributor") against any losses, damages and costs (collectively "Losses") arising from claims, lawsuits and other legal actions brought by a third party against the Indemnified Contributor to the extent caused by the acts or omissions of such Commercial Contributor in connection with its distribution of the Program in a commercial product offering. The obligations in this section do not apply to any claims or Losses relating to any actual or alleged intellectual property infringement. In order to qualify, an Indemnified Contributor must: a) promptly notify the Commercial Contributor in writing of such claim, and b) allow the Commercial Contributor to control, and cooperate with the Commercial Contributor in, the defense and any related settlement negotiations. The Indemnified Contributor may participate in any such claim at its own expense. + +For example, a Contributor might include the Program in a commercial product offering, Product X. That Contributor is then a Commercial Contributor. If that Commercial Contributor then makes performance claims, or offers warranties related to Product X, those performance claims and warranties are such Commercial Contributor's responsibility alone. Under this section, the Commercial Contributor would have to defend claims against the other Contributors related to those performance claims and warranties, and if a court requires any other Contributor to pay any damages as a result, the Commercial Contributor must pay those damages. + +5. NO WARRANTY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF TITLE, NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE. Each Recipient is solely responsible for determining the appropriateness of using and distributing the Program and assumes all risks associated with its exercise of rights under this Agreement , including but not limited to the risks and costs of program errors, compliance with applicable laws, damage to or loss of data, programs or equipment, and unavailability or interruption of operations. + +6. DISCLAIMER OF LIABILITY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. + +7. GENERAL + +If any provision of this Agreement is invalid or unenforceable under applicable law, it shall not affect the validity or enforceability of the remainder of the terms of this Agreement, and without further action by the parties hereto, such provision shall be reformed to the minimum extent necessary to make such provision valid and enforceable. + +If Recipient institutes patent litigation against any entity (including a cross-claim or counterclaim in a lawsuit) alleging that the Program itself (excluding combinations of the Program with other software or hardware) infringes such Recipient's patent(s), then such Recipient's rights granted under Section 2(b) shall terminate as of the date such litigation is filed. + +All Recipient's rights under this Agreement shall terminate if it fails to comply with any of the material terms or conditions of this Agreement and does not cure such failure in a reasonable period of time after becoming aware of such noncompliance. If all Recipient's rights under this Agreement terminate, Recipient agrees to cease use and distribution of the Program as soon as reasonably practicable. However, Recipient's obligations under this Agreement and any licenses granted by Recipient relating to the Program shall continue and survive. + +Everyone is permitted to copy and distribute copies of this Agreement, but in order to avoid inconsistency the Agreement is copyrighted and may only be modified in the following manner. The Agreement Steward reserves the right to publish new versions (including revisions) of this Agreement from time to time. No one other than the Agreement Steward has the right to modify this Agreement. The Eclipse Foundation is the initial Agreement Steward. The Eclipse Foundation may assign the responsibility to serve as the Agreement Steward to a suitable separate entity. Each new version of the Agreement will be given a distinguishing version number. The Program (including Contributions) may always be distributed subject to the version of the Agreement under which it was received. In addition, after a new version of the Agreement is published, Contributor may elect to distribute the Program (including its Contributions) under the new version. Except as expressly stated in Sections 2(a) and 2(b) above, Recipient receives no rights or licenses to the intellectual property of any Contributor under this Agreement, whether expressly, by implication, estoppel or otherwise. All rights in the Program not expressly granted under this Agreement are reserved. + +This Agreement is governed by the laws of the State of New York and the intellectual property laws of the United States of America. No party to this Agreement will bring a legal action under this Agreement more than one year after the cause of action arose. Each party waives its rights to a jury trial in any resulting litigation. + +----- + +The following software may be included in this product: antlr stringtemplate4, antlr runtime. This software contains the following license and notice below: + +[The "BSD license"] +Copyright (c) 2011-2013 Terence Parr +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + + 1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + 2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + 3. The name of the author may not be used to endorse or promote products + derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR +IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES +OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. +IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT +NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF +THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +----- + +The following software may be included in this product: Apache Avro, Apache BVal, Apache Commons BeanUtils Core, Apache Commons CLI, Apache Commons Codec, Apache Commons Configuration, Apache Commons IO, Apache Commons Lang, Apache Commons Logging, Apache Hadoop, Apache Hive, Apache HttpClient, Apache Maven, Apache Thrift, Apache XBean, Bean Validation API, Code Generation Library, Guava, Jackson, Jetty, Joda time, Log4j Implemented Over SLF4J, Ning Asynchronous Http Client, Plexus, Tableau Web Data Connector, airlift, airlift resolver, airlift slice, fastutil, jDBI, javax.inject, jmxutils, jQuery, opencsv, snappy, vis.js. +This software contains the following license and notice below: + + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + +TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + +1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + +2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + +3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + +4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + +5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + +6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + +7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + +8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + +9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + +END OF TERMS AND CONDITIONS + +APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + +Copyright [yyyy] [name of copyright owner] + +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. + +----- + +The following software may be included in this product: asm. This software contains the following license and notice below: + +Copyright (c) 2000-2011 INRIA, France Telecom +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + +1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + +3. Neither the name of the copyright holders nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE +LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN +CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF +THE POSSIBILITY OF SUCH DAMAGE. + +----- + +The following software may be included in this product: findbugs. The source code is available at http://code.google.com/p/findbugs/. You may also request a copy of the source code by sending a request to opensource@fb.com. This software contains the following license and notice below: + + GNU LESSER GENERAL PUBLIC LICENSE + Version 2.1, February 1999 + + Copyright (C) 1991, 1999 Free Software Foundation, Inc. + 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA + Everyone is permitted to copy and distribute verbatim copies + of this license document, but changing it is not allowed. + +[This is the first released version of the Lesser GPL. It also counts + as the successor of the GNU Library Public License, version 2, hence + the version number 2.1.] + + Preamble + + The licenses for most software are designed to take away your +freedom to share and change it. By contrast, the GNU General Public +Licenses are intended to guarantee your freedom to share and change +free software--to make sure the software is free for all its users. + + This license, the Lesser General Public License, applies to some +specially designated software packages--typically libraries--of the +Free Software Foundation and other authors who decide to use it. You +can use it too, but we suggest you first think carefully about whether +this license or the ordinary General Public License is the better +strategy to use in any particular case, based on the explanations below. + + When we speak of free software, we are referring to freedom of use, +not price. Our General Public Licenses are designed to make sure that +you have the freedom to distribute copies of free software (and charge +for this service if you wish); that you receive source code or can get +it if you want it; that you can change the software and use pieces of +it in new free programs; and that you are informed that you can do +these things. + + To protect your rights, we need to make restrictions that forbid +distributors to deny you these rights or to ask you to surrender these +rights. These restrictions translate to certain responsibilities for +you if you distribute copies of the library or if you modify it. + + For example, if you distribute copies of the library, whether gratis +or for a fee, you must give the recipients all the rights that we gave +you. You must make sure that they, too, receive or can get the source +code. If you link other code with the library, you must provide +complete object files to the recipients, so that they can relink them +with the library after making changes to the library and recompiling +it. And you must show them these terms so they know their rights. + + We protect your rights with a two-step method: (1) we copyright the +library, and (2) we offer you this license, which gives you legal +permission to copy, distribute and/or modify the library. + + To protect each distributor, we want to make it very clear that +there is no warranty for the free library. Also, if the library is +modified by someone else and passed on, the recipients should know +that what they have is not the original version, so that the original +author's reputation will not be affected by problems that might be +introduced by others. + + Finally, software patents pose a constant threat to the existence of +any free program. We wish to make sure that a company cannot +effectively restrict the users of a free program by obtaining a +restrictive license from a patent holder. Therefore, we insist that +any patent license obtained for a version of the library must be +consistent with the full freedom of use specified in this license. + + Most GNU software, including some libraries, is covered by the +ordinary GNU General Public License. This license, the GNU Lesser +General Public License, applies to certain designated libraries, and +is quite different from the ordinary General Public License. We use +this license for certain libraries in order to permit linking those +libraries into non-free programs. + + When a program is linked with a library, whether statically or using +a shared library, the combination of the two is legally speaking a +combined work, a derivative of the original library. The ordinary +General Public License therefore permits such linking only if the +entire combination fits its criteria of freedom. The Lesser General +Public License permits more lax criteria for linking other code with +the library. + + We call this license the "Lesser" General Public License because it +does Less to protect the user's freedom than the ordinary General +Public License. It also provides other free software developers Less +of an advantage over competing non-free programs. These disadvantages +are the reason we use the ordinary General Public License for many +libraries. However, the Lesser license provides advantages in certain +special circumstances. + + For example, on rare occasions, there may be a special need to +encourage the widest possible use of a certain library, so that it becomes +a de-facto standard. To achieve this, non-free programs must be +allowed to use the library. A more frequent case is that a free +library does the same job as widely used non-free libraries. In this +case, there is little to gain by limiting the free library to free +software only, so we use the Lesser General Public License. + + In other cases, permission to use a particular library in non-free +programs enables a greater number of people to use a large body of +free software. For example, permission to use the GNU C Library in +non-free programs enables many more people to use the whole GNU +operating system, as well as its variant, the GNU/Linux operating +system. + + Although the Lesser General Public License is Less protective of the +users' freedom, it does ensure that the user of a program that is +linked with the Library has the freedom and the wherewithal to run +that program using a modified version of the Library. + + The precise terms and conditions for copying, distribution and +modification follow. Pay close attention to the difference between a +"work based on the library" and a "work that uses the library". The +former contains code derived from the library, whereas the latter must +be combined with the library in order to run. + + GNU LESSER GENERAL PUBLIC LICENSE + TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + + 0. This License Agreement applies to any software library or other +program which contains a notice placed by the copyright holder or +other authorized party saying it may be distributed under the terms of +this Lesser General Public License (also called "this License"). +Each licensee is addressed as "you". + + A "library" means a collection of software functions and/or data +prepared so as to be conveniently linked with application programs +(which use some of those functions and data) to form executables. + + The "Library", below, refers to any such software library or work +which has been distributed under these terms. A "work based on the +Library" means either the Library or any derivative work under +copyright law: that is to say, a work containing the Library or a +portion of it, either verbatim or with modifications and/or translated +straightforwardly into another language. (Hereinafter, translation is +included without limitation in the term "modification".) + + "Source code" for a work means the preferred form of the work for +making modifications to it. For a library, complete source code means +all the source code for all modules it contains, plus any associated +interface definition files, plus the scripts used to control compilation +and installation of the library. + + Activities other than copying, distribution and modification are not +covered by this License; they are outside its scope. The act of +running a program using the Library is not restricted, and output from +such a program is covered only if its contents constitute a work based +on the Library (independent of the use of the Library in a tool for +writing it). Whether that is true depends on what the Library does +and what the program that uses the Library does. + + 1. You may copy and distribute verbatim copies of the Library's +complete source code as you receive it, in any medium, provided that +you conspicuously and appropriately publish on each copy an +appropriate copyright notice and disclaimer of warranty; keep intact +all the notices that refer to this License and to the absence of any +warranty; and distribute a copy of this License along with the +Library. + + You may charge a fee for the physical act of transferring a copy, +and you may at your option offer warranty protection in exchange for a +fee. + + 2. You may modify your copy or copies of the Library or any portion +of it, thus forming a work based on the Library, and copy and +distribute such modifications or work under the terms of Section 1 +above, provided that you also meet all of these conditions: + + a) The modified work must itself be a software library. + + b) You must cause the files modified to carry prominent notices + stating that you changed the files and the date of any change. + + c) You must cause the whole of the work to be licensed at no + charge to all third parties under the terms of this License. + + d) If a facility in the modified Library refers to a function or a + table of data to be supplied by an application program that uses + the facility, other than as an argument passed when the facility + is invoked, then you must make a good faith effort to ensure that, + in the event an application does not supply such function or + table, the facility still operates, and performs whatever part of + its purpose remains meaningful. + + (For example, a function in a library to compute square roots has + a purpose that is entirely well-defined independent of the + application. Therefore, Subsection 2d requires that any + application-supplied function or table used by this function must + be optional: if the application does not supply it, the square + root function must still compute square roots.) + +These requirements apply to the modified work as a whole. If +identifiable sections of that work are not derived from the Library, +and can be reasonably considered independent and separate works in +themselves, then this License, and its terms, do not apply to those +sections when you distribute them as separate works. But when you +distribute the same sections as part of a whole which is a work based +on the Library, the distribution of the whole must be on the terms of +this License, whose permissions for other licensees extend to the +entire whole, and thus to each and every part regardless of who wrote +it. + +Thus, it is not the intent of this section to claim rights or contest +your rights to work written entirely by you; rather, the intent is to +exercise the right to control the distribution of derivative or +collective works based on the Library. + +In addition, mere aggregation of another work not based on the Library +with the Library (or with a work based on the Library) on a volume of +a storage or distribution medium does not bring the other work under +the scope of this License. + + 3. You may opt to apply the terms of the ordinary GNU General Public +License instead of this License to a given copy of the Library. To do +this, you must alter all the notices that refer to this License, so +that they refer to the ordinary GNU General Public License, version 2, +instead of to this License. (If a newer version than version 2 of the +ordinary GNU General Public License has appeared, then you can specify +that version instead if you wish.) Do not make any other change in +these notices. + + Once this change is made in a given copy, it is irreversible for +that copy, so the ordinary GNU General Public License applies to all +subsequent copies and derivative works made from that copy. + + This option is useful when you wish to copy part of the code of +the Library into a program that is not a library. + + 4. You may copy and distribute the Library (or a portion or +derivative of it, under Section 2) in object code or executable form +under the terms of Sections 1 and 2 above provided that you accompany +it with the complete corresponding machine-readable source code, which +must be distributed under the terms of Sections 1 and 2 above on a +medium customarily used for software interchange. + + If distribution of object code is made by offering access to copy +from a designated place, then offering equivalent access to copy the +source code from the same place satisfies the requirement to +distribute the source code, even though third parties are not +compelled to copy the source along with the object code. + + 5. A program that contains no derivative of any portion of the +Library, but is designed to work with the Library by being compiled or +linked with it, is called a "work that uses the Library". Such a +work, in isolation, is not a derivative work of the Library, and +therefore falls outside the scope of this License. + + However, linking a "work that uses the Library" with the Library +creates an executable that is a derivative of the Library (because it +contains portions of the Library), rather than a "work that uses the +library". The executable is therefore covered by this License. +Section 6 states terms for distribution of such executables. + + When a "work that uses the Library" uses material from a header file +that is part of the Library, the object code for the work may be a +derivative work of the Library even though the source code is not. +Whether this is true is especially significant if the work can be +linked without the Library, or if the work is itself a library. The +threshold for this to be true is not precisely defined by law. + + If such an object file uses only numerical parameters, data +structure layouts and accessors, and small macros and small inline +functions (ten lines or less in length), then the use of the object +file is unrestricted, regardless of whether it is legally a derivative +work. (Executables containing this object code plus portions of the +Library will still fall under Section 6.) + + Otherwise, if the work is a derivative of the Library, you may +distribute the object code for the work under the terms of Section 6. +Any executables containing that work also fall under Section 6, +whether or not they are linked directly with the Library itself. + + 6. As an exception to the Sections above, you may also combine or +link a "work that uses the Library" with the Library to produce a +work containing portions of the Library, and distribute that work +under terms of your choice, provided that the terms permit +modification of the work for the customer's own use and reverse +engineering for debugging such modifications. + + You must give prominent notice with each copy of the work that the +Library is used in it and that the Library and its use are covered by +this License. You must supply a copy of this License. If the work +during execution displays copyright notices, you must include the +copyright notice for the Library among them, as well as a reference +directing the user to the copy of this License. Also, you must do one +of these things: + + a) Accompany the work with the complete corresponding + machine-readable source code for the Library including whatever + changes were used in the work (which must be distributed under + Sections 1 and 2 above); and, if the work is an executable linked + with the Library, with the complete machine-readable "work that + uses the Library", as object code and/or source code, so that the + user can modify the Library and then relink to produce a modified + executable containing the modified Library. (It is understood + that the user who changes the contents of definitions files in the + Library will not necessarily be able to recompile the application + to use the modified definitions.) + + b) Use a suitable shared library mechanism for linking with the + Library. A suitable mechanism is one that (1) uses at run time a + copy of the library already present on the user's computer system, + rather than copying library functions into the executable, and (2) + will operate properly with a modified version of the library, if + the user installs one, as long as the modified version is + interface-compatible with the version that the work was made with. + + c) Accompany the work with a written offer, valid for at + least three years, to give the same user the materials + specified in Subsection 6a, above, for a charge no more + than the cost of performing this distribution. + + d) If distribution of the work is made by offering access to copy + from a designated place, offer equivalent access to copy the above + specified materials from the same place. + + e) Verify that the user has already received a copy of these + materials or that you have already sent this user a copy. + + For an executable, the required form of the "work that uses the +Library" must include any data and utility programs needed for +reproducing the executable from it. However, as a special exception, +the materials to be distributed need not include anything that is +normally distributed (in either source or binary form) with the major +components (compiler, kernel, and so on) of the operating system on +which the executable runs, unless that component itself accompanies +the executable. + + It may happen that this requirement contradicts the license +restrictions of other proprietary libraries that do not normally +accompany the operating system. Such a contradiction means you cannot +use both them and the Library together in an executable that you +distribute. + + 7. You may place library facilities that are a work based on the +Library side-by-side in a single library together with other library +facilities not covered by this License, and distribute such a combined +library, provided that the separate distribution of the work based on +the Library and of the other library facilities is otherwise +permitted, and provided that you do these two things: + + a) Accompany the combined library with a copy of the same work + based on the Library, uncombined with any other library + facilities. This must be distributed under the terms of the + Sections above. + + b) Give prominent notice with the combined library of the fact + that part of it is a work based on the Library, and explaining + where to find the accompanying uncombined form of the same work. + + 8. You may not copy, modify, sublicense, link with, or distribute +the Library except as expressly provided under this License. Any +attempt otherwise to copy, modify, sublicense, link with, or +distribute the Library is void, and will automatically terminate your +rights under this License. However, parties who have received copies, +or rights, from you under this License will not have their licenses +terminated so long as such parties remain in full compliance. + + 9. You are not required to accept this License, since you have not +signed it. However, nothing else grants you permission to modify or +distribute the Library or its derivative works. These actions are +prohibited by law if you do not accept this License. Therefore, by +modifying or distributing the Library (or any work based on the +Library), you indicate your acceptance of this License to do so, and +all its terms and conditions for copying, distributing or modifying +the Library or works based on it. + + 10. Each time you redistribute the Library (or any work based on the +Library), the recipient automatically receives a license from the +original licensor to copy, distribute, link with or modify the Library +subject to these terms and conditions. You may not impose any further +restrictions on the recipients' exercise of the rights granted herein. +You are not responsible for enforcing compliance by third parties with +this License. + + 11. If, as a consequence of a court judgment or allegation of patent +infringement or for any other reason (not limited to patent issues), +conditions are imposed on you (whether by court order, agreement or +otherwise) that contradict the conditions of this License, they do not +excuse you from the conditions of this License. If you cannot +distribute so as to satisfy simultaneously your obligations under this +License and any other pertinent obligations, then as a consequence you +may not distribute the Library at all. For example, if a patent +license would not permit royalty-free redistribution of the Library by +all those who receive copies directly or indirectly through you, then +the only way you could satisfy both it and this License would be to +refrain entirely from distribution of the Library. + +If any portion of this section is held invalid or unenforceable under any +particular circumstance, the balance of the section is intended to apply, +and the section as a whole is intended to apply in other circumstances. + +It is not the purpose of this section to induce you to infringe any +patents or other property right claims or to contest validity of any +such claims; this section has the sole purpose of protecting the +integrity of the free software distribution system which is +implemented by public license practices. Many people have made +generous contributions to the wide range of software distributed +through that system in reliance on consistent application of that +system; it is up to the author/donor to decide if he or she is willing +to distribute software through any other system and a licensee cannot +impose that choice. + +This section is intended to make thoroughly clear what is believed to +be a consequence of the rest of this License. + + 12. If the distribution and/or use of the Library is restricted in +certain countries either by patents or by copyrighted interfaces, the +original copyright holder who places the Library under this License may add +an explicit geographical distribution limitation excluding those countries, +so that distribution is permitted only in or among countries not thus +excluded. In such case, this License incorporates the limitation as if +written in the body of this License. + + 13. The Free Software Foundation may publish revised and/or new +versions of the Lesser General Public License from time to time. +Such new versions will be similar in spirit to the present version, +but may differ in detail to address new problems or concerns. + +Each version is given a distinguishing version number. If the Library +specifies a version number of this License which applies to it and +"any later version", you have the option of following the terms and +conditions either of that version or of any later version published by +the Free Software Foundation. If the Library does not specify a +license version number, you may choose any version ever published by +the Free Software Foundation. + + 14. If you wish to incorporate parts of the Library into other free +programs whose distribution conditions are incompatible with these, +write to the author to ask for permission. For software which is +copyrighted by the Free Software Foundation, write to the Free +Software Foundation; we sometimes make exceptions for this. Our +decision will be guided by the two goals of preserving the free status +of all derivatives of our free software and of promoting the sharing +and reuse of software generally. + + NO WARRANTY + + 15. BECAUSE THE LIBRARY IS LICENSED FREE OF CHARGE, THERE IS NO +WARRANTY FOR THE LIBRARY, TO THE EXTENT PERMITTED BY APPLICABLE LAW. +EXCEPT WHEN OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR +OTHER PARTIES PROVIDE THE LIBRARY "AS IS" WITHOUT WARRANTY OF ANY +KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR +PURPOSE. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE +LIBRARY IS WITH YOU. SHOULD THE LIBRARY PROVE DEFECTIVE, YOU ASSUME +THE COST OF ALL NECESSARY SERVICING, REPAIR OR CORRECTION. + + 16. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN +WRITING WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY +AND/OR REDISTRIBUTE THE LIBRARY AS PERMITTED ABOVE, BE LIABLE TO YOU +FOR DAMAGES, INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR +CONSEQUENTIAL DAMAGES ARISING OUT OF THE USE OR INABILITY TO USE THE +LIBRARY (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA BEING +RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A +FAILURE OF THE LIBRARY TO OPERATE WITH ANY OTHER SOFTWARE), EVEN IF +SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH +DAMAGES. + + END OF TERMS AND CONDITIONS + + How to Apply These Terms to Your New Libraries + + If you develop a new library, and you want it to be of the greatest +possible use to the public, we recommend making it free software that +everyone can redistribute and change. You can do so by permitting +redistribution under these terms (or, alternatively, under the terms of the +ordinary General Public License). + + To apply these terms, attach the following notices to the library. It is +safest to attach them to the start of each source file to most effectively +convey the exclusion of warranty; and each file should have at least the +"copyright" line and a pointer to where the full notice is found. + + + Copyright (C) + + This library is free software; you can redistribute it and/or + modify it under the terms of the GNU Lesser General Public + License as published by the Free Software Foundation; either + version 2.1 of the License, or (at your option) any later version. + + This library is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU + Lesser General Public License for more details. + + You should have received a copy of the GNU Lesser General Public + License along with this library; if not, write to the Free Software + Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA + +Also add information on how to contact you by electronic and paper mail. + +You should also get your employer (if you work as a programmer) or your +school, if any, to sign a "copyright disclaimer" for the library, if +necessary. Here is a sample; alter the names: + + Yoyodyne, Inc., hereby disclaims all copyright interest in the + library `Frob' (a library for tweaking knobs) written by James Random Hacker. + + , 1 April 1990 + Ty Coon, President of Vice + +That's all there is to it! + +-- From LICENSE-ASM.txt: + +Copyright (c) 2000-2005 INRIA, France Telecom +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + +1. Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright +notice, this list of conditions and the following disclaimer in the +documentation and/or other materials provided with the distribution. + +3. Neither the name of the copyright holders nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE +LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN +CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF +THE POSSIBILITY OF SUCH DAMAGE. + +-- From LICENSE-AppleJavaExtensions.txt: + +AppleJavaExtensions +v 1.2 + +This is a pluggable jar of stub classes representing the new Apple eAWT and eIO APIs for Java 1.4 on Mac OS X. The purpose of these stubs is to allow for compilation of eAWT- or eIO-referencing code on platforms other than Mac OS X. The jar file is enclosed in a zip archive for easy expansion on other platforms. + +These stubs are not intended for the runtime classpath on non-Mac platforms. Please see the OSXAdapter sample for how to write cross-platform code that uses eAWT. + +Disclaimer: IMPORTANT: This Apple software is supplied to you by Apple +Computer, Inc. ("Apple") in consideration of your agreement to the +following terms, and your use, installation, modification or +redistribution of this Apple software constitutes acceptance of these +terms. If you do not agree with these terms, please do not use, +install, modify or redistribute this Apple software. + +In consideration of your agreement to abide by the following terms, and +subject to these terms, Apple grants you a personal, non-exclusive +license, under Apple's copyrights in this original Apple software (the +"Apple Software"), to use, reproduce, modify and redistribute the Apple +Software, with or without modifications, in source and/or binary forms; +provided that if you redistribute the Apple Software in its entirety and +without modifications, you must retain this notice and the following +text and disclaimers in all such redistributions of the Apple Software. +Neither the name, trademarks, service marks or logos of Apple Computer, +Inc. may be used to endorse or promote products derived from the Apple +Software without specific prior written permission from Apple. Except +as expressly stated in this notice, no other rights or licenses, express +or implied, are granted by Apple herein, including but not limited to +any patent rights that may be infringed by your derivative works or by +other works in which the Apple Software may be incorporated. + +The Apple Software is provided by Apple on an "AS IS" basis. APPLE +MAKES NO WARRANTIES, EXPRESS OR IMPLIED, INCLUDING WITHOUT LIMITATION +THE IMPLIED WARRANTIES OF NON-INFRINGEMENT, MERCHANTABILITY AND FITNESS +FOR A PARTICULAR PURPOSE, REGARDING THE APPLE SOFTWARE OR ITS USE AND +OPERATION ALONE OR IN COMBINATION WITH YOUR PRODUCTS. + +IN NO EVENT SHALL APPLE BE LIABLE FOR ANY SPECIAL, INDIRECT, INCIDENTAL +OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +INTERRUPTION) ARISING IN ANY WAY OUT OF THE USE, REPRODUCTION, +MODIFICATION AND/OR DISTRIBUTION OF THE APPLE SOFTWARE, HOWEVER CAUSED +AND WHETHER UNDER THEORY OF CONTRACT, TORT (INCLUDING NEGLIGENCE), +STRICT LIABILITY OR OTHERWISE, EVEN IF APPLE HAS BEEN ADVISED OF THE +POSSIBILITY OF SUCH DAMAGE. + +Copyright © 2003-2006 Apple Computer, Inc., All Rights Reserved + +-- From LICENSE-bcel.txt: + +/* + * Apache License + * Version 2.0, January 2004 + * http://www.apache.org/licenses/ + * + * TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + * + * 1. Definitions. + * + * "License" shall mean the terms and conditions for use, reproduction, + * and distribution as defined by Sections 1 through 9 of this document. + * + * "Licensor" shall mean the copyright owner or entity authorized by + * the copyright owner that is granting the License. + * + * "Legal Entity" shall mean the union of the acting entity and all + * other entities that control, are controlled by, or are under common + * control with that entity. For the purposes of this definition, + * "control" means (i) the power, direct or indirect, to cause the + * direction or management of such entity, whether by contract or + * otherwise, or (ii) ownership of fifty percent (50%) or more of the + * outstanding shares, or (iii) beneficial ownership of such entity. + * + * "You" (or "Your") shall mean an individual or Legal Entity + * exercising permissions granted by this License. + * + * "Source" form shall mean the preferred form for making modifications, + * including but not limited to software source code, documentation + * source, and configuration files. + * + * "Object" form shall mean any form resulting from mechanical + * transformation or translation of a Source form, including but + * not limited to compiled object code, generated documentation, + * and conversions to other media types. + * + * "Work" shall mean the work of authorship, whether in Source or + * Object form, made available under the License, as indicated by a + * copyright notice that is included in or attached to the work + * (an example is provided in the Appendix below). + * + * "Derivative Works" shall mean any work, whether in Source or Object + * form, that is based on (or derived from) the Work and for which the + * editorial revisions, annotations, elaborations, or other modifications + * represent, as a whole, an original work of authorship. For the purposes + * of this License, Derivative Works shall not include works that remain + * separable from, or merely link (or bind by name) to the interfaces of, + * the Work and Derivative Works thereof. + * + * "Contribution" shall mean any work of authorship, including + * the original version of the Work and any modifications or additions + * to that Work or Derivative Works thereof, that is intentionally + * submitted to Licensor for inclusion in the Work by the copyright owner + * or by an individual or Legal Entity authorized to submit on behalf of + * the copyright owner. For the purposes of this definition, "submitted" + * means any form of electronic, verbal, or written communication sent + * to the Licensor or its representatives, including but not limited to + * communication on electronic mailing lists, source code control systems, + * and issue tracking systems that are managed by, or on behalf of, the + * Licensor for the purpose of discussing and improving the Work, but + * excluding communication that is conspicuously marked or otherwise + * designated in writing by the copyright owner as "Not a Contribution." + * + * "Contributor" shall mean Licensor and any individual or Legal Entity + * on behalf of whom a Contribution has been received by Licensor and + * subsequently incorporated within the Work. + * + * 2. Grant of Copyright License. Subject to the terms and conditions of + * this License, each Contributor hereby grants to You a perpetual, + * worldwide, non-exclusive, no-charge, royalty-free, irrevocable + * copyright license to reproduce, prepare Derivative Works of, + * publicly display, publicly perform, sublicense, and distribute the + * Work and such Derivative Works in Source or Object form. + * + * 3. Grant of Patent License. Subject to the terms and conditions of + * this License, each Contributor hereby grants to You a perpetual, + * worldwide, non-exclusive, no-charge, royalty-free, irrevocable + * (except as stated in this section) patent license to make, have made, + * use, offer to sell, sell, import, and otherwise transfer the Work, + * where such license applies only to those patent claims licensable + * by such Contributor that are necessarily infringed by their + * Contribution(s) alone or by combination of their Contribution(s) + * with the Work to which such Contribution(s) was submitted. If You + * institute patent litigation against any entity (including a + * cross-claim or counterclaim in a lawsuit) alleging that the Work + * or a Contribution incorporated within the Work constitutes direct + * or contributory patent infringement, then any patent licenses + * granted to You under this License for that Work shall terminate + * as of the date such litigation is filed. + * + * 4. Redistribution. You may reproduce and distribute copies of the + * Work or Derivative Works thereof in any medium, with or without + * modifications, and in Source or Object form, provided that You + * meet the following conditions: + * + * (a) You must give any other recipients of the Work or + * Derivative Works a copy of this License; and + * + * (b) You must cause any modified files to carry prominent notices + * stating that You changed the files; and + * + * (c) You must retain, in the Source form of any Derivative Works + * that You distribute, all copyright, patent, trademark, and + * attribution notices from the Source form of the Work, + * excluding those notices that do not pertain to any part of + * the Derivative Works; and + * + * (d) If the Work includes a "NOTICE" text file as part of its + * distribution, then any Derivative Works that You distribute must + * include a readable copy of the attribution notices contained + * within such NOTICE file, excluding those notices that do not + * pertain to any part of the Derivative Works, in at least one + * of the following places: within a NOTICE text file distributed + * as part of the Derivative Works; within the Source form or + * documentation, if provided along with the Derivative Works; or, + * within a display generated by the Derivative Works, if and + * wherever such third-party notices normally appear. The contents + * of the NOTICE file are for informational purposes only and + * do not modify the License. You may add Your own attribution + * notices within Derivative Works that You distribute, alongside + * or as an addendum to the NOTICE text from the Work, provided + * that such additional attribution notices cannot be construed + * as modifying the License. + * + * You may add Your own copyright statement to Your modifications and + * may provide additional or different license terms and conditions + * for use, reproduction, or distribution of Your modifications, or + * for any such Derivative Works as a whole, provided Your use, + * reproduction, and distribution of the Work otherwise complies with + * the conditions stated in this License. + * + * 5. Submission of Contributions. Unless You explicitly state otherwise, + * any Contribution intentionally submitted for inclusion in the Work + * by You to the Licensor shall be under the terms and conditions of + * this License, without any additional terms or conditions. + * Notwithstanding the above, nothing herein shall supersede or modify + * the terms of any separate license agreement you may have executed + * with Licensor regarding such Contributions. + * + * 6. Trademarks. This License does not grant permission to use the trade + * names, trademarks, service marks, or product names of the Licensor, + * except as required for reasonable and customary use in describing the + * origin of the Work and reproducing the content of the NOTICE file. + * + * 7. Disclaimer of Warranty. Unless required by applicable law or + * agreed to in writing, Licensor provides the Work (and each + * Contributor provides its Contributions) on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + * implied, including, without limitation, any warranties or conditions + * of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + * PARTICULAR PURPOSE. You are solely responsible for determining the + * appropriateness of using or redistributing the Work and assume any + * risks associated with Your exercise of permissions under this License. + * + * 8. Limitation of Liability. In no event and under no legal theory, + * whether in tort (including negligence), contract, or otherwise, + * unless required by applicable law (such as deliberate and grossly + * negligent acts) or agreed to in writing, shall any Contributor be + * liable to You for damages, including any direct, indirect, special, + * incidental, or consequential damages of any character arising as a + * result of this License or out of the use or inability to use the + * Work (including but not limited to damages for loss of goodwill, + * work stoppage, computer failure or malfunction, or any and all + * other commercial damages or losses), even if such Contributor + * has been advised of the possibility of such damages. + * + * 9. Accepting Warranty or Additional Liability. While redistributing + * the Work or Derivative Works thereof, You may choose to offer, + * and charge a fee for, acceptance of support, warranty, indemnity, + * or other liability obligations and/or rights consistent with this + * License. However, in accepting such obligations, You may act only + * on Your own behalf and on Your sole responsibility, not on behalf + * of any other Contributor, and only if You agree to indemnify, + * defend, and hold each Contributor harmless for any liability + * incurred by, or claims asserted against, such Contributor by reason + * of your accepting any such warranty or additional liability. + * + * END OF TERMS AND CONDITIONS + * + * APPENDIX: How to apply the Apache License to your work. + * + * To apply the Apache License to your work, attach the following + * boilerplate notice, with the fields enclosed by brackets "[]" + * replaced with your own identifying information. (Don't include + * the brackets!) The text should be enclosed in the appropriate + * comment syntax for the file format. We also recommend that a + * file or class name and description of purpose be included on the + * same "printed page" as the copyright notice for easier + * identification within third-party archives. + * + * Copyright [yyyy] [name of copyright owner] + * + * 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. + */ + +-- From LICENSE-commons-lang.txt: + + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + +TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + +1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + +2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + +3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + +4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + +5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + +6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + +7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + +8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + +9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + +END OF TERMS AND CONDITIONS + +APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + +Copyright [yyyy] [name of copyright owner] + +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. + +-- From LICENSE-docbook.txt: + + + +-- From LICENSE-dom4j.txt: + +BSD style license + +Redistribution and use of this software and associated documentation +("Software"), with or without modification, are permitted provided that +the following conditions are met: + +1. Redistributions of source code must retain copyright statements +and notices. Redistributions must also contain a copy of this +document. + +2. Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following +disclaimer in the documentation and/or other materials provided +with the distribution. + +3. The name "DOM4J" must not be used to endorse or promote +products derived from this Software without prior written +permission of MetaStuff, Ltd. For written permission, please +contact dom4j-info@metastuff.com. + +4. Products derived from this Software may not be called "DOM4J" +nor may "DOM4J" appear in their names without prior written +permission of MetaStuff, Ltd. DOM4J is a registered trademark of +MetaStuff, Ltd. + +5. Due credit should be given to the DOM4J Project +(http://dom4j.org/). + +THIS SOFTWARE IS PROVIDED BY METASTUFF, LTD. AND CONTRIBUTORS ``AS IS'' +AND ANY EXPRESSED OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, +THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR +PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL METASTUFF, LTD. OR ITS +CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, +EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR +PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +Copyright 2001 (C) MetaStuff, Ltd. All Rights Reserved. + +-- From LICENSE-jFormatString.txt: + +The GNU General Public License (GPL) + +Version 2, June 1991 + +Copyright (C) 1989, 1991 Free Software Foundation, Inc. +59 Temple Place, Suite 330, Boston, MA 02111-1307 USA + +Everyone is permitted to copy and distribute verbatim copies of this license +document, but changing it is not allowed. + +Preamble + +The licenses for most software are designed to take away your freedom to share +and change it. By contrast, the GNU General Public License is intended to +guarantee your freedom to share and change free software--to make sure the +software is free for all its users. This General Public License applies to +most of the Free Software Foundation's software and to any other program whose +authors commit to using it. (Some other Free Software Foundation software is +covered by the GNU Library General Public License instead.) You can apply it to +your programs, too. + +When we speak of free software, we are referring to freedom, not price. Our +General Public Licenses are designed to make sure that you have the freedom to +distribute copies of free software (and charge for this service if you wish), +that you receive source code or can get it if you want it, that you can change +the software or use pieces of it in new free programs; and that you know you +can do these things. + +To protect your rights, we need to make restrictions that forbid anyone to deny +you these rights or to ask you to surrender the rights. These restrictions +translate to certain responsibilities for you if you distribute copies of the +software, or if you modify it. + +For example, if you distribute copies of such a program, whether gratis or for +a fee, you must give the recipients all the rights that you have. You must +make sure that they, too, receive or can get the source code. And you must +show them these terms so they know their rights. + +We protect your rights with two steps: (1) copyright the software, and (2) +offer you this license which gives you legal permission to copy, distribute +and/or modify the software. + +Also, for each author's protection and ours, we want to make certain that +everyone understands that there is no warranty for this free software. If the +software is modified by someone else and passed on, we want its recipients to +know that what they have is not the original, so that any problems introduced +by others will not reflect on the original authors' reputations. + +Finally, any free program is threatened constantly by software patents. We +wish to avoid the danger that redistributors of a free program will +individually obtain patent licenses, in effect making the program proprietary. +To prevent this, we have made it clear that any patent must be licensed for +everyone's free use or not licensed at all. + +The precise terms and conditions for copying, distribution and modification +follow. + +TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + +0. This License applies to any program or other work which contains a notice +placed by the copyright holder saying it may be distributed under the terms of +this General Public License. The "Program", below, refers to any such program +or work, and a "work based on the Program" means either the Program or any +derivative work under copyright law: that is to say, a work containing the +Program or a portion of it, either verbatim or with modifications and/or +translated into another language. (Hereinafter, translation is included +without limitation in the term "modification".) Each licensee is addressed as +"you". + +Activities other than copying, distribution and modification are not covered by +this License; they are outside its scope. The act of running the Program is +not restricted, and the output from the Program is covered only if its contents +constitute a work based on the Program (independent of having been made by +running the Program). Whether that is true depends on what the Program does. + +1. You may copy and distribute verbatim copies of the Program's source code as +you receive it, in any medium, provided that you conspicuously and +appropriately publish on each copy an appropriate copyright notice and +disclaimer of warranty; keep intact all the notices that refer to this License +and to the absence of any warranty; and give any other recipients of the +Program a copy of this License along with the Program. + +You may charge a fee for the physical act of transferring a copy, and you may +at your option offer warranty protection in exchange for a fee. + +2. You may modify your copy or copies of the Program or any portion of it, thus +forming a work based on the Program, and copy and distribute such modifications +or work under the terms of Section 1 above, provided that you also meet all of +these conditions: + +a) You must cause the modified files to carry prominent notices stating +that you changed the files and the date of any change. + +b) You must cause any work that you distribute or publish, that in whole or +in part contains or is derived from the Program or any part thereof, to be +licensed as a whole at no charge to all third parties under the terms of +this License. + +c) If the modified program normally reads commands interactively when run, +you must cause it, when started running for such interactive use in the +most ordinary way, to print or display an announcement including an +appropriate copyright notice and a notice that there is no warranty (or +else, saying that you provide a warranty) and that users may redistribute +the program under these conditions, and telling the user how to view a copy +of this License. (Exception: if the Program itself is interactive but does +not normally print such an announcement, your work based on the Program is +not required to print an announcement.) + +These requirements apply to the modified work as a whole. If identifiable +sections of that work are not derived from the Program, and can be reasonably +considered independent and separate works in themselves, then this License, and +its terms, do not apply to those sections when you distribute them as separate +works. But when you distribute the same sections as part of a whole which is a +work based on the Program, the distribution of the whole must be on the terms +of this License, whose permissions for other licensees extend to the entire +whole, and thus to each and every part regardless of who wrote it. + +Thus, it is not the intent of this section to claim rights or contest your +rights to work written entirely by you; rather, the intent is to exercise the +right to control the distribution of derivative or collective works based on +the Program. + +In addition, mere aggregation of another work not based on the Program with the +Program (or with a work based on the Program) on a volume of a storage or +distribution medium does not bring the other work under the scope of this +License. + +3. You may copy and distribute the Program (or a work based on it, under +Section 2) in object code or executable form under the terms of Sections 1 and +2 above provided that you also do one of the following: + +a) Accompany it with the complete corresponding machine-readable source +code, which must be distributed under the terms of Sections 1 and 2 above +on a medium customarily used for software interchange; or, + +b) Accompany it with a written offer, valid for at least three years, to +give any third party, for a charge no more than your cost of physically +performing source distribution, a complete machine-readable copy of the +corresponding source code, to be distributed under the terms of Sections 1 +and 2 above on a medium customarily used for software interchange; or, + +c) Accompany it with the information you received as to the offer to +distribute corresponding source code. (This alternative is allowed only +for noncommercial distribution and only if you received the program in +object code or executable form with such an offer, in accord with +Subsection b above.) + +The source code for a work means the preferred form of the work for making +modifications to it. For an executable work, complete source code means all +the source code for all modules it contains, plus any associated interface +definition files, plus the scripts used to control compilation and installation +of the executable. However, as a special exception, the source code +distributed need not include anything that is normally distributed (in either +source or binary form) with the major components (compiler, kernel, and so on) +of the operating system on which the executable runs, unless that component +itself accompanies the executable. + +If distribution of executable or object code is made by offering access to copy +from a designated place, then offering equivalent access to copy the source +code from the same place counts as distribution of the source code, even though +third parties are not compelled to copy the source along with the object code. + +4. You may not copy, modify, sublicense, or distribute the Program except as +expressly provided under this License. Any attempt otherwise to copy, modify, +sublicense or distribute the Program is void, and will automatically terminate +your rights under this License. However, parties who have received copies, or +rights, from you under this License will not have their licenses terminated so +long as such parties remain in full compliance. + +5. You are not required to accept this License, since you have not signed it. +However, nothing else grants you permission to modify or distribute the Program +or its derivative works. These actions are prohibited by law if you do not +accept this License. Therefore, by modifying or distributing the Program (or +any work based on the Program), you indicate your acceptance of this License to +do so, and all its terms and conditions for copying, distributing or modifying +the Program or works based on it. + +6. Each time you redistribute the Program (or any work based on the Program), +the recipient automatically receives a license from the original licensor to +copy, distribute or modify the Program subject to these terms and conditions. +You may not impose any further restrictions on the recipients' exercise of the +rights granted herein. You are not responsible for enforcing compliance by +third parties to this License. + +7. If, as a consequence of a court judgment or allegation of patent +infringement or for any other reason (not limited to patent issues), conditions +are imposed on you (whether by court order, agreement or otherwise) that +contradict the conditions of this License, they do not excuse you from the +conditions of this License. If you cannot distribute so as to satisfy +simultaneously your obligations under this License and any other pertinent +obligations, then as a consequence you may not distribute the Program at all. +For example, if a patent license would not permit royalty-free redistribution +of the Program by all those who receive copies directly or indirectly through +you, then the only way you could satisfy both it and this License would be to +refrain entirely from distribution of the Program. + +If any portion of this section is held invalid or unenforceable under any +particular circumstance, the balance of the section is intended to apply and +the section as a whole is intended to apply in other circumstances. + +It is not the purpose of this section to induce you to infringe any patents or +other property right claims or to contest validity of any such claims; this +section has the sole purpose of protecting the integrity of the free software +distribution system, which is implemented by public license practices. Many +people have made generous contributions to the wide range of software +distributed through that system in reliance on consistent application of that +system; it is up to the author/donor to decide if he or she is willing to +distribute software through any other system and a licensee cannot impose that +choice. + +This section is intended to make thoroughly clear what is believed to be a +consequence of the rest of this License. + +8. If the distribution and/or use of the Program is restricted in certain +countries either by patents or by copyrighted interfaces, the original +copyright holder who places the Program under this License may add an explicit +geographical distribution limitation excluding those countries, so that +distribution is permitted only in or among countries not thus excluded. In +such case, this License incorporates the limitation as if written in the body +of this License. + +9. The Free Software Foundation may publish revised and/or new versions of the +General Public License from time to time. Such new versions will be similar in +spirit to the present version, but may differ in detail to address new problems +or concerns. + +Each version is given a distinguishing version number. If the Program +specifies a version number of this License which applies to it and "any later +version", you have the option of following the terms and conditions either of +that version or of any later version published by the Free Software Foundation. +If the Program does not specify a version number of this License, you may +choose any version ever published by the Free Software Foundation. + +10. If you wish to incorporate parts of the Program into other free programs +whose distribution conditions are different, write to the author to ask for +permission. For software which is copyrighted by the Free Software Foundation, +write to the Free Software Foundation; we sometimes make exceptions for this. +Our decision will be guided by the two goals of preserving the free status of +all derivatives of our free software and of promoting the sharing and reuse of +software generally. + +NO WARRANTY + +11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO WARRANTY FOR +THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. EXCEPT WHEN OTHERWISE +STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR OTHER PARTIES PROVIDE THE +PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, +INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND +FITNESS FOR A PARTICULAR PURPOSE. THE ENTIRE RISK AS TO THE QUALITY AND +PERFORMANCE OF THE PROGRAM IS WITH YOU. SHOULD THE PROGRAM PROVE DEFECTIVE, +YOU ASSUME THE COST OF ALL NECESSARY SERVICING, REPAIR OR CORRECTION. + +12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING WILL +ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY AND/OR REDISTRIBUTE THE +PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, INCLUDING ANY +GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING OUT OF THE USE OR +INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED TO LOSS OF DATA OR DATA +BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY YOU OR THIRD PARTIES OR A +FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER PROGRAMS), EVEN IF SUCH HOLDER +OR OTHER PARTY HAS BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. + +END OF TERMS AND CONDITIONS + +How to Apply These Terms to Your New Programs + +If you develop a new program, and you want it to be of the greatest possible +use to the public, the best way to achieve this is to make it free software +which everyone can redistribute and change under these terms. + +To do so, attach the following notices to the program. It is safest to attach +them to the start of each source file to most effectively convey the exclusion +of warranty; and each file should have at least the "copyright" line and a +pointer to where the full notice is found. + +One line to give the program's name and a brief idea of what it does. + +Copyright (C) + +This program is free software; you can redistribute it and/or modify it +under the terms of the GNU General Public License as published by the Free +Software Foundation; either version 2 of the License, or (at your option) +any later version. + +This program is distributed in the hope that it will be useful, but WITHOUT +ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or +FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License for +more details. + +You should have received a copy of the GNU General Public License along +with this program; if not, write to the Free Software Foundation, Inc., 59 +Temple Place, Suite 330, Boston, MA 02111-1307 USA + +Also add information on how to contact you by electronic and paper mail. + +If the program is interactive, make it output a short notice like this when it +starts in an interactive mode: + +Gnomovision version 69, Copyright (C) year name of author Gnomovision comes +with ABSOLUTELY NO WARRANTY; for details type 'show w'. This is free +software, and you are welcome to redistribute it under certain conditions; +type 'show c' for details. + +The hypothetical commands 'show w' and 'show c' should show the appropriate +parts of the General Public License. Of course, the commands you use may be +called something other than 'show w' and 'show c'; they could even be +mouse-clicks or menu items--whatever suits your program. + +You should also get your employer (if you work as a programmer) or your school, +if any, to sign a "copyright disclaimer" for the program, if necessary. Here +is a sample; alter the names: + +Yoyodyne, Inc., hereby disclaims all copyright interest in the program +'Gnomovision' (which makes passes at compilers) written by James Hacker. + +signature of Ty Coon, 1 April 1989 + +Ty Coon, President of Vice + +This General Public License does not permit incorporating your program into +proprietary programs. If your program is a subroutine library, you may +consider it more useful to permit linking proprietary applications with the +library. If this is what you want to do, use the GNU Library General Public +License instead of this License. + + +"CLASSPATH" EXCEPTION TO THE GPL + +Certain source files distributed by Sun Microsystems, Inc. are subject to +the following clarification and special exception to the GPL, but only where +Sun has expressly included in the particular source file's header the words +"Sun designates this particular file as subject to the "Classpath" exception +as provided by Sun in the LICENSE file that accompanied this code." + +Linking this library statically or dynamically with other modules is making +a combined work based on this library. Thus, the terms and conditions of +the GNU General Public License cover the whole combination. + +As a special exception, the copyright holders of this library give you +permission to link this library with independent modules to produce an +executable, regardless of the license terms of these independent modules, +and to copy and distribute the resulting executable under terms of your +choice, provided that you also meet, for each linked independent module, +the terms and conditions of the license of that module. An independent +module is a module which is not derived from or based on this library. If +you modify this library, you may extend this exception to your version of +the library, but you are not obligated to do so. If you do not wish to do +so, delete this exception statement from your version. + +-- From LICENSE-jaxen.txt: + +/* +$Id: LICENSE-jaxen.txt,v 1.1 2008/06/18 18:54:23 wpugh Exp $ + +Copyright 2003-2006 The Werken Company. All Rights Reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + +* Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright +notice, this list of conditions and the following disclaimer in the +documentation and/or other materials provided with the distribution. + +* Neither the name of the Jaxen Project nor the names of its +contributors may be used to endorse or promote products derived +from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS +IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED +TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A +PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER +OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, +EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR +PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +*/ + +-- From LICENSE-jcip.txt: + +The Java code in the package net.jcip.annotations +is copyright (c) 2005 Brian Goetz +and is released under the Creative Commons Attribution License +(http://creativecommons.org/licenses/by/2.5) +Official home: http://www.jcip.net + +-- From LICENSE-jdepend.txt: + +The jdepend library (lib/jdepend-2.9.jar) is distributed under the terms of the BSD license: +http://www.clarkware.com/software/JDepend.html#license +http://www.clarkware.com/software/license.txt + +Copyright (C) 2001 Clarkware Consulting, Inc. +All Rights Reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + +1. Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright +notice, this list of conditions and the following disclaimer in the +documentation and/or other materials provided with the distribution. + +3. Neither the name of Clarkware Consulting, Inc. nor the names of its +contributors may be used to endorse or promote products derived +from this software without prior written permission. For written +permission, please contact clarkware@clarkware.com. + +THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED WARRANTIES, +INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND +FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL +CLARKWARE CONSULTING OR ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, +OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, +EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-- From LICENSE-jsr305.txt: + +The JSR-305 reference implementation (lib/jsr305.jar) is +distributed under the terms of the New BSD license: + +http://www.opensource.org/licenses/bsd-license.php + +See the JSR-305 home page for more information: + +http://code.google.com/p/jsr-305/ + +----- + +The following software may be included in this product: floatingdecimal. The source code is available at https://github.com/airlift/floatingdecimal. You may also request a copy of the source code by sending a request to opensource@fb.com. This software contains the following license and notice below: + +/* +* Copyright (c) 1996, 2011, Oracle and/or its affiliates. All rights reserved. +* DO NOT ALTER OR REMOVE COPYRIGHT NOTICES OR THIS FILE HEADER. +* +* This code is free software; you can redistribute it and/or modify it +* under the terms of the GNU General Public License version 2 only, as +* published by the Free Software Foundation. Oracle designates this +* particular file as subject to the "Classpath" exception as provided +* by Oracle in the LICENSE file that accompanied this code. +* +* This code is distributed in the hope that it will be useful, but WITHOUT +* ANY WARRANTY; without even the implied warranty of MERCHANTABILITY or +* FITNESS FOR A PARTICULAR PURPOSE. See the GNU General Public License +* version 2 for more details (a copy is included in the LICENSE file that +* accompanied this code). +* +* You should have received a copy of the GNU General Public License version +* 2 along with this work; if not, write to the Free Software Foundation, +* Inc., 51 Franklin St, Fifth Floor, Boston, MA 02110-1301 USA. +* +* Please contact Oracle, 500 Oracle Parkway, Redwood Shores, CA 94065 USA +* or visit www.oracle.com if you need additional information or have any +* questions. +*/ + + GNU GENERAL PUBLIC LICENSE + Version 2, June 1991 + + Copyright (C) 1989, 1991 Free Software Foundation, Inc., + 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA + Everyone is permitted to copy and distribute verbatim copies + of this license document, but changing it is not allowed. + + Preamble + + The licenses for most software are designed to take away your +freedom to share and change it. By contrast, the GNU General Public +License is intended to guarantee your freedom to share and change free +software--to make sure the software is free for all its users. This +General Public License applies to most of the Free Software +Foundation's software and to any other program whose authors commit to +using it. (Some other Free Software Foundation software is covered by +the GNU Lesser General Public License instead.) You can apply it to +your programs, too. + + When we speak of free software, we are referring to freedom, not +price. Our General Public Licenses are designed to make sure that you +have the freedom to distribute copies of free software (and charge for +this service if you wish), that you receive source code or can get it +if you want it, that you can change the software or use pieces of it +in new free programs; and that you know you can do these things. + + To protect your rights, we need to make restrictions that forbid +anyone to deny you these rights or to ask you to surrender the rights. +These restrictions translate to certain responsibilities for you if you +distribute copies of the software, or if you modify it. + + For example, if you distribute copies of such a program, whether +gratis or for a fee, you must give the recipients all the rights that +you have. You must make sure that they, too, receive or can get the +source code. And you must show them these terms so they know their +rights. + + We protect your rights with two steps: (1) copyright the software, and +(2) offer you this license which gives you legal permission to copy, +distribute and/or modify the software. + + Also, for each author's protection and ours, we want to make certain +that everyone understands that there is no warranty for this free +software. If the software is modified by someone else and passed on, we +want its recipients to know that what they have is not the original, so +that any problems introduced by others will not reflect on the original +authors' reputations. + + Finally, any free program is threatened constantly by software +patents. We wish to avoid the danger that redistributors of a free +program will individually obtain patent licenses, in effect making the +program proprietary. To prevent this, we have made it clear that any +patent must be licensed for everyone's free use or not licensed at all. + + The precise terms and conditions for copying, distribution and +modification follow. + + GNU GENERAL PUBLIC LICENSE + TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + + 0. This License applies to any program or other work which contains +a notice placed by the copyright holder saying it may be distributed +under the terms of this General Public License. The "Program", below, +refers to any such program or work, and a "work based on the Program" +means either the Program or any derivative work under copyright law: +that is to say, a work containing the Program or a portion of it, +either verbatim or with modifications and/or translated into another +language. (Hereinafter, translation is included without limitation in +the term "modification".) Each licensee is addressed as "you". + +Activities other than copying, distribution and modification are not +covered by this License; they are outside its scope. The act of +running the Program is not restricted, and the output from the Program +is covered only if its contents constitute a work based on the +Program (independent of having been made by running the Program). +Whether that is true depends on what the Program does. + + 1. You may copy and distribute verbatim copies of the Program's +source code as you receive it, in any medium, provided that you +conspicuously and appropriately publish on each copy an appropriate +copyright notice and disclaimer of warranty; keep intact all the +notices that refer to this License and to the absence of any warranty; +and give any other recipients of the Program a copy of this License +along with the Program. + +You may charge a fee for the physical act of transferring a copy, and +you may at your option offer warranty protection in exchange for a fee. + + 2. You may modify your copy or copies of the Program or any portion +of it, thus forming a work based on the Program, and copy and +distribute such modifications or work under the terms of Section 1 +above, provided that you also meet all of these conditions: + + a) You must cause the modified files to carry prominent notices + stating that you changed the files and the date of any change. + + b) You must cause any work that you distribute or publish, that in + whole or in part contains or is derived from the Program or any + part thereof, to be licensed as a whole at no charge to all third + parties under the terms of this License. + + c) If the modified program normally reads commands interactively + when run, you must cause it, when started running for such + interactive use in the most ordinary way, to print or display an + announcement including an appropriate copyright notice and a + notice that there is no warranty (or else, saying that you provide + a warranty) and that users may redistribute the program under + these conditions, and telling the user how to view a copy of this + License. (Exception: if the Program itself is interactive but + does not normally print such an announcement, your work based on + the Program is not required to print an announcement.) + +These requirements apply to the modified work as a whole. If +identifiable sections of that work are not derived from the Program, +and can be reasonably considered independent and separate works in +themselves, then this License, and its terms, do not apply to those +sections when you distribute them as separate works. But when you +distribute the same sections as part of a whole which is a work based +on the Program, the distribution of the whole must be on the terms of +this License, whose permissions for other licensees extend to the +entire whole, and thus to each and every part regardless of who wrote it. + +Thus, it is not the intent of this section to claim rights or contest +your rights to work written entirely by you; rather, the intent is to +exercise the right to control the distribution of derivative or +collective works based on the Program. + +In addition, mere aggregation of another work not based on the Program +with the Program (or with a work based on the Program) on a volume of +a storage or distribution medium does not bring the other work under +the scope of this License. + + 3. You may copy and distribute the Program (or a work based on it, +under Section 2) in object code or executable form under the terms of +Sections 1 and 2 above provided that you also do one of the following: + + a) Accompany it with the complete corresponding machine-readable + source code, which must be distributed under the terms of Sections + 1 and 2 above on a medium customarily used for software interchange; or, + + b) Accompany it with a written offer, valid for at least three + years, to give any third party, for a charge no more than your + cost of physically performing source distribution, a complete + machine-readable copy of the corresponding source code, to be + distributed under the terms of Sections 1 and 2 above on a medium + customarily used for software interchange; or, + + c) Accompany it with the information you received as to the offer + to distribute corresponding source code. (This alternative is + allowed only for noncommercial distribution and only if you + received the program in object code or executable form with such + an offer, in accord with Subsection b above.) + +The source code for a work means the preferred form of the work for +making modifications to it. For an executable work, complete source +code means all the source code for all modules it contains, plus any +associated interface definition files, plus the scripts used to +control compilation and installation of the executable. However, as a +special exception, the source code distributed need not include +anything that is normally distributed (in either source or binary +form) with the major components (compiler, kernel, and so on) of the +operating system on which the executable runs, unless that component +itself accompanies the executable. + +If distribution of executable or object code is made by offering +access to copy from a designated place, then offering equivalent +access to copy the source code from the same place counts as +distribution of the source code, even though third parties are not +compelled to copy the source along with the object code. + + 4. You may not copy, modify, sublicense, or distribute the Program +except as expressly provided under this License. Any attempt +otherwise to copy, modify, sublicense or distribute the Program is +void, and will automatically terminate your rights under this License. +However, parties who have received copies, or rights, from you under +this License will not have their licenses terminated so long as such +parties remain in full compliance. + + 5. You are not required to accept this License, since you have not +signed it. However, nothing else grants you permission to modify or +distribute the Program or its derivative works. These actions are +prohibited by law if you do not accept this License. Therefore, by +modifying or distributing the Program (or any work based on the +Program), you indicate your acceptance of this License to do so, and +all its terms and conditions for copying, distributing or modifying +the Program or works based on it. + + 6. Each time you redistribute the Program (or any work based on the +Program), the recipient automatically receives a license from the +original licensor to copy, distribute or modify the Program subject to +these terms and conditions. You may not impose any further +restrictions on the recipients' exercise of the rights granted herein. +You are not responsible for enforcing compliance by third parties to +this License. + + 7. If, as a consequence of a court judgment or allegation of patent +infringement or for any other reason (not limited to patent issues), +conditions are imposed on you (whether by court order, agreement or +otherwise) that contradict the conditions of this License, they do not +excuse you from the conditions of this License. If you cannot +distribute so as to satisfy simultaneously your obligations under this +License and any other pertinent obligations, then as a consequence you +may not distribute the Program at all. For example, if a patent +license would not permit royalty-free redistribution of the Program by +all those who receive copies directly or indirectly through you, then +the only way you could satisfy both it and this License would be to +refrain entirely from distribution of the Program. + +If any portion of this section is held invalid or unenforceable under +any particular circumstance, the balance of the section is intended to +apply and the section as a whole is intended to apply in other +circumstances. + +It is not the purpose of this section to induce you to infringe any +patents or other property right claims or to contest validity of any +such claims; this section has the sole purpose of protecting the +integrity of the free software distribution system, which is +implemented by public license practices. Many people have made +generous contributions to the wide range of software distributed +through that system in reliance on consistent application of that +system; it is up to the author/donor to decide if he or she is willing +to distribute software through any other system and a licensee cannot +impose that choice. + +This section is intended to make thoroughly clear what is believed to +be a consequence of the rest of this License. + + 8. If the distribution and/or use of the Program is restricted in +certain countries either by patents or by copyrighted interfaces, the +original copyright holder who places the Program under this License +may add an explicit geographical distribution limitation excluding +those countries, so that distribution is permitted only in or among +countries not thus excluded. In such case, this License incorporates +the limitation as if written in the body of this License. + + 9. The Free Software Foundation may publish revised and/or new versions +of the General Public License from time to time. Such new versions will +be similar in spirit to the present version, but may differ in detail to +address new problems or concerns. + +Each version is given a distinguishing version number. If the Program +specifies a version number of this License which applies to it and "any +later version", you have the option of following the terms and conditions +either of that version or of any later version published by the Free +Software Foundation. If the Program does not specify a version number of +this License, you may choose any version ever published by the Free Software +Foundation. + + 10. If you wish to incorporate parts of the Program into other free +programs whose distribution conditions are different, write to the author +to ask for permission. For software which is copyrighted by the Free +Software Foundation, write to the Free Software Foundation; we sometimes +make exceptions for this. Our decision will be guided by the two goals +of preserving the free status of all derivatives of our free software and +of promoting the sharing and reuse of software generally. + + NO WARRANTY + + 11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO WARRANTY +FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. EXCEPT WHEN +OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR OTHER PARTIES +PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED +OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE ENTIRE RISK AS +TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH YOU. SHOULD THE +PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL NECESSARY SERVICING, +REPAIR OR CORRECTION. + + 12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING +WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY AND/OR +REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, +INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING +OUT OF THE USE OR INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED +TO LOSS OF DATA OR DATA BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY +YOU OR THIRD PARTIES OR A FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER +PROGRAMS), EVEN IF SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE +POSSIBILITY OF SUCH DAMAGES. + + END OF TERMS AND CONDITIONS + + How to Apply These Terms to Your New Programs + + If you develop a new program, and you want it to be of the greatest +possible use to the public, the best way to achieve this is to make it +free software which everyone can redistribute and change under these terms. + + To do so, attach the following notices to the program. It is safest +to attach them to the start of each source file to most effectively +convey the exclusion of warranty; and each file should have at least +the "copyright" line and a pointer to where the full notice is found. + + + Copyright (C) + + This program is free software; you can redistribute it and/or modify + it under the terms of the GNU General Public License as published by + the Free Software Foundation; either version 2 of the License, or + (at your option) any later version. + + This program is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + GNU General Public License for more details. + + You should have received a copy of the GNU General Public License along + with this program; if not, write to the Free Software Foundation, Inc., + 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. + +Also add information on how to contact you by electronic and paper mail. + +If the program is interactive, make it output a short notice like this +when it starts in an interactive mode: + + Gnomovision version 69, Copyright (C) year name of author + Gnomovision comes with ABSOLUTELY NO WARRANTY; for details type `show w'. + This is free software, and you are welcome to redistribute it + under certain conditions; type `show c' for details. + +The hypothetical commands `show w' and `show c' should show the appropriate +parts of the General Public License. Of course, the commands you use may +be called something other than `show w' and `show c'; they could even be +mouse-clicks or menu items--whatever suits your program. + +You should also get your employer (if you work as a programmer) or your +school, if any, to sign a "copyright disclaimer" for the program, if +necessary. Here is a sample; alter the names: + + Yoyodyne, Inc., hereby disclaims all copyright interest in the program + `Gnomovision' (which makes passes at compilers) written by James Hacker. + + , 1 April 1989 + Ty Coon, President of Vice + +This General Public License does not permit incorporating your program into +proprietary programs. If your program is a subroutine library, you may +consider it more useful to permit linking proprietary applications with the +library. If this is what you want to do, use the GNU Lesser General +Public License instead of this License. + +----- + +The following software may be included in this product: H2 Database Engine. The source code is available at http://www.h2database.com/html/download.html. You may also request a copy of the source code by sending a request to opensource@fb.com. This software contains the following license and notice below: + +Eclipse Public License - v 1.0 + +THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. + +1. DEFINITIONS + +"Contribution" means: + +a) in the case of the initial Contributor, the initial code and documentation distributed under this Agreement, and + +b) in the case of each subsequent Contributor: + +i) changes to the Program, and + +ii) additions to the Program; + +where such changes and/or additions to the Program originate from and are distributed by that particular Contributor. A Contribution 'originates' from a Contributor if it was added to the Program by such Contributor itself or anyone acting on such Contributor's behalf. Contributions do not include additions to the Program which: (i) are separate modules of software distributed in conjunction with the Program under their own license agreement, and (ii) are not derivative works of the Program. + +"Contributor" means any person or entity that distributes the Program. + +"Licensed Patents" mean patent claims licensable by a Contributor which are necessarily infringed by the use or sale of its Contribution alone or when combined with the Program. + +"Program" means the Contributions distributed in accordance with this Agreement. + +"Recipient" means anyone who receives the Program under this Agreement, including all Contributors. + +2. GRANT OF RIGHTS + +a) Subject to the terms of this Agreement, each Contributor hereby grants Recipient a non-exclusive, worldwide, royalty-free copyright license to reproduce, prepare derivative works of, publicly display, publicly perform, distribute and sublicense the Contribution of such Contributor, if any, and such derivative works, in source code and object code form. + +b) Subject to the terms of this Agreement, each Contributor hereby grants Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed Patents to make, use, sell, offer to sell, import and otherwise transfer the Contribution of such Contributor, if any, in source code and object code form. This patent license shall apply to the combination of the Contribution and the Program if, at the time the Contribution is added by the Contributor, such addition of the Contribution causes such combination to be covered by the Licensed Patents. The patent license shall not apply to any other combinations which include the Contribution. No hardware per se is licensed hereunder. + +c) Recipient understands that although each Contributor grants the licenses to its Contributions set forth herein, no assurances are provided by any Contributor that the Program does not infringe the patent or other intellectual property rights of any other entity. Each Contributor disclaims any liability to Recipient for claims brought by any other entity based on infringement of intellectual property rights or otherwise. As a condition to exercising the rights and licenses granted hereunder, each Recipient hereby assumes sole responsibility to secure any other intellectual property rights needed, if any. For example, if a third party patent license is required to allow Recipient to distribute the Program, it is Recipient's responsibility to acquire that license before distributing the Program. + +d) Each Contributor represents that to its knowledge it has sufficient copyright rights in its Contribution, if any, to grant the copyright license set forth in this Agreement. + +3. REQUIREMENTS + +A Contributor may choose to distribute the Program in object code form under its own license agreement, provided that: + +a) it complies with the terms and conditions of this Agreement; and + +b) its license agreement: + +i) effectively disclaims on behalf of all Contributors all warranties and conditions, express and implied, including warranties or conditions of title and non-infringement, and implied warranties or conditions of merchantability and fitness for a particular purpose; + +ii) effectively excludes on behalf of all Contributors all liability for damages, including direct, indirect, special, incidental and consequential damages, such as lost profits; + +iii) states that any provisions which differ from this Agreement are offered by that Contributor alone and not by any other party; and + +iv) states that source code for the Program is available from such Contributor, and informs licensees how to obtain it in a reasonable manner on or through a medium customarily used for software exchange. + +When the Program is made available in source code form: + +a) it must be made available under this Agreement; and + +b) a copy of this Agreement must be included with each copy of the Program. + +Contributors may not remove or alter any copyright notices contained within the Program. + +Each Contributor must identify itself as the originator of its Contribution, if any, in a manner that reasonably allows subsequent Recipients to identify the originator of the Contribution. + +4. COMMERCIAL DISTRIBUTION + +Commercial distributors of software may accept certain responsibilities with respect to end users, business partners and the like. While this license is intended to facilitate the commercial use of the Program, the Contributor who includes the Program in a commercial product offering should do so in a manner which does not create potential liability for other Contributors. Therefore, if a Contributor includes the Program in a commercial product offering, such Contributor ("Commercial Contributor") hereby agrees to defend and indemnify every other Contributor ("Indemnified Contributor") against any losses, damages and costs (collectively "Losses") arising from claims, lawsuits and other legal actions brought by a third party against the Indemnified Contributor to the extent caused by the acts or omissions of such Commercial Contributor in connection with its distribution of the Program in a commercial product offering. The obligations in this section do not apply to any claims or Losses relating to any actual or alleged intellectual property infringement. In order to qualify, an Indemnified Contributor must: a) promptly notify the Commercial Contributor in writing of such claim, and b) allow the Commercial Contributor to control, and cooperate with the Commercial Contributor in, the defense and any related settlement negotiations. The Indemnified Contributor may participate in any such claim at its own expense. + +For example, a Contributor might include the Program in a commercial product offering, Product X. That Contributor is then a Commercial Contributor. If that Commercial Contributor then makes performance claims, or offers warranties related to Product X, those performance claims and warranties are such Commercial Contributor's responsibility alone. Under this section, the Commercial Contributor would have to defend claims against the other Contributors related to those performance claims and warranties, and if a court requires any other Contributor to pay any damages as a result, the Commercial Contributor must pay those damages. + +5. NO WARRANTY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF TITLE, NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE. Each Recipient is solely responsible for determining the appropriateness of using and distributing the Program and assumes all risks associated with its exercise of rights under this Agreement , including but not limited to the risks and costs of program errors, compliance with applicable laws, damage to or loss of data, programs or equipment, and unavailability or interruption of operations. + +6. DISCLAIMER OF LIABILITY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. + +7. GENERAL + +If any provision of this Agreement is invalid or unenforceable under applicable law, it shall not affect the validity or enforceability of the remainder of the terms of this Agreement, and without further action by the parties hereto, such provision shall be reformed to the minimum extent necessary to make such provision valid and enforceable. + +If Recipient institutes patent litigation against any entity (including a cross-claim or counterclaim in a lawsuit) alleging that the Program itself (excluding combinations of the Program with other software or hardware) infringes such Recipient's patent(s), then such Recipient's rights granted under Section 2(b) shall terminate as of the date such litigation is filed. + +All Recipient's rights under this Agreement shall terminate if it fails to comply with any of the material terms or conditions of this Agreement and does not cure such failure in a reasonable period of time after becoming aware of such noncompliance. If all Recipient's rights under this Agreement terminate, Recipient agrees to cease use and distribution of the Program as soon as reasonably practicable. However, Recipient's obligations under this Agreement and any licenses granted by Recipient relating to the Program shall continue and survive. + +Everyone is permitted to copy and distribute copies of this Agreement, but in order to avoid inconsistency the Agreement is copyrighted and may only be modified in the following manner. The Agreement Steward reserves the right to publish new versions (including revisions) of this Agreement from time to time. No one other than the Agreement Steward has the right to modify this Agreement. The Eclipse Foundation is the initial Agreement Steward. The Eclipse Foundation may assign the responsibility to serve as the Agreement Steward to a suitable separate entity. Each new version of the Agreement will be given a distinguishing version number. The Program (including Contributions) may always be distributed subject to the version of the Agreement under which it was received. In addition, after a new version of the Agreement is published, Contributor may elect to distribute the Program (including its Contributions) under the new version. Except as expressly stated in Sections 2(a) and 2(b) above, Recipient receives no rights or licenses to the intellectual property of any Contributor under this Agreement, whether expressly, by implication, estoppel or otherwise. All rights in the Program not expressly granted under this Agreement are reserved. + +This Agreement is governed by the laws of the State of New York and the intellectual property laws of the United States of America. No party to this Agreement will bring a legal action under this Agreement more than one year after the cause of action arose. Each party waives its rights to a jury trial in any resulting litigation. + +----- + +The following software may be included in this product: java servlet api. The source code is available at https://java.net/projects/servlet-spec/. You may also request a copy of the source code by sending a request to opensource@fb.com. This software contains the following license and notice below: + +COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL - Version 1.1) +1. Definitions. + + 1.1. “Contributor” means each individual or entity that creates or contributes to the creation of Modifications. + + 1.2. “Contributor Version” means the combination of the Original Software, prior Modifications used by a Contributor (if any), and the Modifications made by that particular Contributor. + + 1.3. “Covered Software” means (a) the Original Software, or (b) Modifications, or (c) the combination of files containing Original Software with files containing Modifications, in each case including portions thereof. + + 1.4. “Executable” means the Covered Software in any form other than Source Code. + + 1.5. “Initial Developer” means the individual or entity that first makes Original Software available under this License. + + 1.6. “Larger Work” means a work which combines Covered Software or portions thereof with code not governed by the terms of this License. + + 1.7. “License” means this document. + + 1.8. “Licensable” means having the right to grant, to the maximum extent possible, whether at the time of the initial grant or subsequently acquired, any and all of the rights conveyed herein. + + 1.9. “Modifications” means the Source Code and Executable form of any of the following: + + A. Any file that results from an addition to, deletion from or modification of the contents of a file containing Original Software or previous Modifications; + + B. Any new file that contains any part of the Original Software or previous Modification; or + + C. Any new file that is contributed or otherwise made available under the terms of this License. + + 1.10. “Original Software” means the Source Code and Executable form of computer software code that is originally released under this License. + + 1.11. “Patent Claims” means any patent claim(s), now owned or hereafter acquired, including without limitation, method, process, and apparatus claims, in any patent Licensable by grantor. + + 1.12. “Source Code” means (a) the common form of computer software code in which modifications are made and (b) associated documentation included in or with such code. + + 1.13. “You” (or “Your”) means an individual or a legal entity exercising rights under, and complying with all of the terms of, this License. For legal entities, “You” includes any entity which controls, is controlled by, or is under common control with You. For purposes of this definition, “control” means (a) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (b) ownership of more than fifty percent (50%) of the outstanding shares or beneficial ownership of such entity. + +2. License Grants. + + 2.1. The Initial Developer Grant. + + Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, the Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive license: + + (a) under intellectual property rights (other than patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, display, perform, sublicense and distribute the Original Software (or portions thereof), with or without Modifications, and/or as part of a Larger Work; and + + (b) under Patent Claims infringed by the making, using or selling of Original Software, to make, have made, use, practice, sell, and offer for sale, and/or otherwise dispose of the Original Software (or portions thereof). + + (c) The licenses granted in Sections 2.1(a) and (b) are effective on the date Initial Developer first distributes or otherwise makes the Original Software available to a third party under the terms of this License. + + (d) Notwithstanding Section 2.1(b) above, no patent license is granted: (1) for code that You delete from the Original Software, or (2) for infringements caused by: (i) the modification of the Original Software, or (ii) the combination of the Original Software with other software or devices. + + 2.2. Contributor Grant. + + Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, each Contributor hereby grants You a world-wide, royalty-free, non-exclusive license: + + (a) under intellectual property rights (other than patent or trademark) Licensable by Contributor to use, reproduce, modify, display, perform, sublicense and distribute the Modifications created by such Contributor (or portions thereof), either on an unmodified basis, with other Modifications, as Covered Software and/or as part of a Larger Work; and + + (b) under Patent Claims infringed by the making, using, or selling of Modifications made by that Contributor either alone and/or in combination with its Contributor Version (or portions of such combination), to make, use, sell, offer for sale, have made, and/or otherwise dispose of: (1) Modifications made by that Contributor (or portions thereof); and (2) the combination of Modifications made by that Contributor with its Contributor Version (or portions of such combination). + + (c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first distributes or otherwise makes the Modifications available to a third party. + + (d) Notwithstanding Section 2.2(b) above, no patent license is granted: (1) for any code that Contributor has deleted from the Contributor Version; (2) for infringements caused by: (i) third party modifications of Contributor Version, or (ii) the combination of Modifications made by that Contributor with other software (except as part of the Contributor Version) or other devices; or (3) under Patent Claims infringed by Covered Software in the absence of Modifications made by that Contributor. + +3. Distribution Obligations. + + 3.1. Availability of Source Code. + + Any Covered Software that You distribute or otherwise make available in Executable form must also be made available in Source Code form and that Source Code form must be distributed only under the terms of this License. You must include a copy of this License with every copy of the Source Code form of the Covered Software You distribute or otherwise make available. You must inform recipients of any such Covered Software in Executable form as to how they can obtain such Covered Software in Source Code form in a reasonable manner on or through a medium customarily used for software exchange. + + 3.2. Modifications. + + The Modifications that You create or to which You contribute are governed by the terms of this License. You represent that You believe Your Modifications are Your original creation(s) and/or You have sufficient rights to grant the rights conveyed by this License. + + 3.3. Required Notices. + + You must include a notice in each of Your Modifications that identifies You as the Contributor of the Modification. You may not remove or alter any copyright, patent or trademark notices contained within the Covered Software, or any notices of licensing or any descriptive text giving attribution to any Contributor or the Initial Developer. + + 3.4. Application of Additional Terms. + + You may not offer or impose any terms on any Covered Software in Source Code form that alters or restricts the applicable version of this License or the recipients’ rights hereunder. You may choose to offer, and to charge a fee for, warranty, support, indemnity or liability obligations to one or more recipients of Covered Software. However, you may do so only on Your own behalf, and not on behalf of the Initial Developer or any Contributor. You must make it absolutely clear that any such warranty, support, indemnity or liability obligation is offered by You alone, and You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of warranty, support, indemnity or liability terms You offer. + + 3.5. Distribution of Executable Versions. + + You may distribute the Executable form of the Covered Software under the terms of this License or under the terms of a license of Your choice, which may contain terms different from this License, provided that You are in compliance with the terms of this License and that the license for the Executable form does not attempt to limit or alter the recipient’s rights in the Source Code form from the rights set forth in this License. If You distribute the Covered Software in Executable form under a different license, You must make it absolutely clear that any terms which differ from this License are offered by You alone, not by the Initial Developer or Contributor. You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of any such terms You offer. + + 3.6. Larger Works. + + You may create a Larger Work by combining Covered Software with other code not governed by the terms of this License and distribute the Larger Work as a single product. In such a case, You must make sure the requirements of this License are fulfilled for the Covered Software. + +4. Versions of the License. + + 4.1. New Versions. + + Oracle is the initial license steward and may publish revised and/or new versions of this License from time to time. Each version will be given a distinguishing version number. Except as provided in Section 4.3, no one other than the license steward has the right to modify this License. + + 4.2. Effect of New Versions. + + You may always continue to use, distribute or otherwise make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. If the Initial Developer includes a notice in the Original Software prohibiting it from being distributed or otherwise made available under any subsequent version of the License, You must distribute and make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. Otherwise, You may also choose to use, distribute or otherwise make the Covered Software available under the terms of any subsequent version of the License published by the license steward. + + 4.3. Modified Versions. + + When You are an Initial Developer and You want to create a new license for Your Original Software, You may create and use a modified version of this License if You: (a) rename the license and remove any references to the name of the license steward (except to note that the license differs from this License); and (b) otherwise make it clear that the license contains terms which differ from this License. + +5. DISCLAIMER OF WARRANTY. + +COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN “AS IS” BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. +6. TERMINATION. + + 6.1. This License and the rights granted hereunder will terminate automatically if You fail to comply with terms herein and fail to cure such breach within 30 days of becoming aware of the breach. Provisions which, by their nature, must remain in effect beyond the termination of this License shall survive. + + 6.2. If You assert a patent infringement claim (excluding declaratory judgment actions) against Initial Developer or a Contributor (the Initial Developer or Contributor against whom You assert such claim is referred to as “Participant”) alleging that the Participant Software (meaning the Contributor Version where the Participant is a Contributor or the Original Software where the Participant is the Initial Developer) directly or indirectly infringes any patent, then any and all rights granted directly or indirectly to You by such Participant, the Initial Developer (if the Initial Developer is not the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from Participant terminate prospectively and automatically at the expiration of such 60 day notice period, unless if within such 60 day period You withdraw Your claim with respect to the Participant Software against such Participant either unilaterally or pursuant to a written agreement with Participant. + + 6.3. If You assert a patent infringement claim against Participant alleging that the Participant Software directly or indirectly infringes any patent where such claim is resolved (such as by license or settlement) prior to the initiation of patent infringement litigation, then the reasonable value of the licenses granted by such Participant under Sections 2.1 or 2.2 shall be taken into account in determining the amount or value of any payment or license. + + 6.4. In the event of termination under Sections 6.1 or 6.2 above, all end user licenses that have been validly granted by You or any distributor hereunder prior to termination (excluding licenses granted to You by any distributor) shall survive termination. + +7. LIMITATION OF LIABILITY. + +UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTY’S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU. +8. U.S. GOVERNMENT END USERS. + +The Covered Software is a “commercial item,” as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of “commercial computer software” (as that term is defined at 48 C.F.R. § 252.227-7014(a)(1)) and “commercial computer software documentation” as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software with only those rights set forth herein. This U.S. Government Rights clause is in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision that addresses Government rights in computer software under this License. +9. MISCELLANEOUS. + +This License represents the complete agreement concerning subject matter hereof. If any provision of this License is held to be unenforceable, such provision shall be reformed only to the extent necessary to make it enforceable. This License shall be governed by the law of the jurisdiction specified in a notice contained within the Original Software (except to the extent applicable law, if any, provides otherwise), excluding such jurisdiction’s conflict-of-law provisions. Any litigation relating to this License shall be subject to the jurisdiction of the courts located in the jurisdiction and venue specified in a notice contained within the Original Software, with the losing party responsible for costs, including, without limitation, court costs and reasonable attorneys’ fees and expenses. The application of the United Nations Convention on Contracts for the International Sale of Goods is expressly excluded. Any law or regulation which provides that the language of a contract shall be construed against the drafter shall not apply to this License. You agree that You alone are responsible for compliance with the United States export administration regulations (and the export control laws and regulation of any other countries) when You use, distribute or otherwise make available any Covered Software. +10. RESPONSIBILITY FOR CLAIMS. + +As between Initial Developer and the Contributors, each party is responsible for claims and damages arising, directly or indirectly, out of its utilization of rights under this License and You agree to work with Initial Developer and Contributors to distribute such responsibility on an equitable basis. Nothing herein is intended or shall be deemed to constitute any admission of liability. + +----- + +The following software may be included in this product: JCodings. This software contains the following license and notice below: + +Permission is hereby granted, free of charge, to any person obtaining a copy of +this software and associated documentation files (the "Software"), to deal in +the Software without restriction, including without limitation the rights to +use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies +of the Software, and to permit persons to whom the Software is furnished to do +so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +----- + +The following software may be included in this product: jersey. The source code is available at https://jersey.java.net/download.html. You may also request a copy of the source code by sending a request to opensource@fb.com. This software contains the following license and notice below: + +COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL - Version 1.1) +1. Definitions. + + 1.1. “Contributor” means each individual or entity that creates or contributes to the creation of Modifications. + + 1.2. “Contributor Version” means the combination of the Original Software, prior Modifications used by a Contributor (if any), and the Modifications made by that particular Contributor. + + 1.3. “Covered Software” means (a) the Original Software, or (b) Modifications, or (c) the combination of files containing Original Software with files containing Modifications, in each case including portions thereof. + + 1.4. “Executable” means the Covered Software in any form other than Source Code. + + 1.5. “Initial Developer” means the individual or entity that first makes Original Software available under this License. + + 1.6. “Larger Work” means a work which combines Covered Software or portions thereof with code not governed by the terms of this License. + + 1.7. “License” means this document. + + 1.8. “Licensable” means having the right to grant, to the maximum extent possible, whether at the time of the initial grant or subsequently acquired, any and all of the rights conveyed herein. + + 1.9. “Modifications” means the Source Code and Executable form of any of the following: + + A. Any file that results from an addition to, deletion from or modification of the contents of a file containing Original Software or previous Modifications; + + B. Any new file that contains any part of the Original Software or previous Modification; or + + C. Any new file that is contributed or otherwise made available under the terms of this License. + + 1.10. “Original Software” means the Source Code and Executable form of computer software code that is originally released under this License. + + 1.11. “Patent Claims” means any patent claim(s), now owned or hereafter acquired, including without limitation, method, process, and apparatus claims, in any patent Licensable by grantor. + + 1.12. “Source Code” means (a) the common form of computer software code in which modifications are made and (b) associated documentation included in or with such code. + + 1.13. “You” (or “Your”) means an individual or a legal entity exercising rights under, and complying with all of the terms of, this License. For legal entities, “You” includes any entity which controls, is controlled by, or is under common control with You. For purposes of this definition, “control” means (a) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (b) ownership of more than fifty percent (50%) of the outstanding shares or beneficial ownership of such entity. + +2. License Grants. + + 2.1. The Initial Developer Grant. + + Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, the Initial Developer hereby grants You a world-wide, royalty-free, non-exclusive license: + + (a) under intellectual property rights (other than patent or trademark) Licensable by Initial Developer, to use, reproduce, modify, display, perform, sublicense and distribute the Original Software (or portions thereof), with or without Modifications, and/or as part of a Larger Work; and + + (b) under Patent Claims infringed by the making, using or selling of Original Software, to make, have made, use, practice, sell, and offer for sale, and/or otherwise dispose of the Original Software (or portions thereof). + + (c) The licenses granted in Sections 2.1(a) and (b) are effective on the date Initial Developer first distributes or otherwise makes the Original Software available to a third party under the terms of this License. + + (d) Notwithstanding Section 2.1(b) above, no patent license is granted: (1) for code that You delete from the Original Software, or (2) for infringements caused by: (i) the modification of the Original Software, or (ii) the combination of the Original Software with other software or devices. + + 2.2. Contributor Grant. + + Conditioned upon Your compliance with Section 3.1 below and subject to third party intellectual property claims, each Contributor hereby grants You a world-wide, royalty-free, non-exclusive license: + + (a) under intellectual property rights (other than patent or trademark) Licensable by Contributor to use, reproduce, modify, display, perform, sublicense and distribute the Modifications created by such Contributor (or portions thereof), either on an unmodified basis, with other Modifications, as Covered Software and/or as part of a Larger Work; and + + (b) under Patent Claims infringed by the making, using, or selling of Modifications made by that Contributor either alone and/or in combination with its Contributor Version (or portions of such combination), to make, use, sell, offer for sale, have made, and/or otherwise dispose of: (1) Modifications made by that Contributor (or portions thereof); and (2) the combination of Modifications made by that Contributor with its Contributor Version (or portions of such combination). + + (c) The licenses granted in Sections 2.2(a) and 2.2(b) are effective on the date Contributor first distributes or otherwise makes the Modifications available to a third party. + + (d) Notwithstanding Section 2.2(b) above, no patent license is granted: (1) for any code that Contributor has deleted from the Contributor Version; (2) for infringements caused by: (i) third party modifications of Contributor Version, or (ii) the combination of Modifications made by that Contributor with other software (except as part of the Contributor Version) or other devices; or (3) under Patent Claims infringed by Covered Software in the absence of Modifications made by that Contributor. + +3. Distribution Obligations. + + 3.1. Availability of Source Code. + + Any Covered Software that You distribute or otherwise make available in Executable form must also be made available in Source Code form and that Source Code form must be distributed only under the terms of this License. You must include a copy of this License with every copy of the Source Code form of the Covered Software You distribute or otherwise make available. You must inform recipients of any such Covered Software in Executable form as to how they can obtain such Covered Software in Source Code form in a reasonable manner on or through a medium customarily used for software exchange. + + 3.2. Modifications. + + The Modifications that You create or to which You contribute are governed by the terms of this License. You represent that You believe Your Modifications are Your original creation(s) and/or You have sufficient rights to grant the rights conveyed by this License. + + 3.3. Required Notices. + + You must include a notice in each of Your Modifications that identifies You as the Contributor of the Modification. You may not remove or alter any copyright, patent or trademark notices contained within the Covered Software, or any notices of licensing or any descriptive text giving attribution to any Contributor or the Initial Developer. + + 3.4. Application of Additional Terms. + + You may not offer or impose any terms on any Covered Software in Source Code form that alters or restricts the applicable version of this License or the recipients’ rights hereunder. You may choose to offer, and to charge a fee for, warranty, support, indemnity or liability obligations to one or more recipients of Covered Software. However, you may do so only on Your own behalf, and not on behalf of the Initial Developer or any Contributor. You must make it absolutely clear that any such warranty, support, indemnity or liability obligation is offered by You alone, and You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of warranty, support, indemnity or liability terms You offer. + + 3.5. Distribution of Executable Versions. + + You may distribute the Executable form of the Covered Software under the terms of this License or under the terms of a license of Your choice, which may contain terms different from this License, provided that You are in compliance with the terms of this License and that the license for the Executable form does not attempt to limit or alter the recipient’s rights in the Source Code form from the rights set forth in this License. If You distribute the Covered Software in Executable form under a different license, You must make it absolutely clear that any terms which differ from this License are offered by You alone, not by the Initial Developer or Contributor. You hereby agree to indemnify the Initial Developer and every Contributor for any liability incurred by the Initial Developer or such Contributor as a result of any such terms You offer. + + 3.6. Larger Works. + + You may create a Larger Work by combining Covered Software with other code not governed by the terms of this License and distribute the Larger Work as a single product. In such a case, You must make sure the requirements of this License are fulfilled for the Covered Software. + +4. Versions of the License. + + 4.1. New Versions. + + Oracle is the initial license steward and may publish revised and/or new versions of this License from time to time. Each version will be given a distinguishing version number. Except as provided in Section 4.3, no one other than the license steward has the right to modify this License. + + 4.2. Effect of New Versions. + + You may always continue to use, distribute or otherwise make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. If the Initial Developer includes a notice in the Original Software prohibiting it from being distributed or otherwise made available under any subsequent version of the License, You must distribute and make the Covered Software available under the terms of the version of the License under which You originally received the Covered Software. Otherwise, You may also choose to use, distribute or otherwise make the Covered Software available under the terms of any subsequent version of the License published by the license steward. + + 4.3. Modified Versions. + + When You are an Initial Developer and You want to create a new license for Your Original Software, You may create and use a modified version of this License if You: (a) rename the license and remove any references to the name of the license steward (except to note that the license differs from this License); and (b) otherwise make it clear that the license contains terms which differ from this License. + +5. DISCLAIMER OF WARRANTY. + +COVERED SOFTWARE IS PROVIDED UNDER THIS LICENSE ON AN “AS IS” BASIS, WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED OR IMPLIED, INCLUDING, WITHOUT LIMITATION, WARRANTIES THAT THE COVERED SOFTWARE IS FREE OF DEFECTS, MERCHANTABLE, FIT FOR A PARTICULAR PURPOSE OR NON-INFRINGING. THE ENTIRE RISK AS TO THE QUALITY AND PERFORMANCE OF THE COVERED SOFTWARE IS WITH YOU. SHOULD ANY COVERED SOFTWARE PROVE DEFECTIVE IN ANY RESPECT, YOU (NOT THE INITIAL DEVELOPER OR ANY OTHER CONTRIBUTOR) ASSUME THE COST OF ANY NECESSARY SERVICING, REPAIR OR CORRECTION. THIS DISCLAIMER OF WARRANTY CONSTITUTES AN ESSENTIAL PART OF THIS LICENSE. NO USE OF ANY COVERED SOFTWARE IS AUTHORIZED HEREUNDER EXCEPT UNDER THIS DISCLAIMER. +6. TERMINATION. + + 6.1. This License and the rights granted hereunder will terminate automatically if You fail to comply with terms herein and fail to cure such breach within 30 days of becoming aware of the breach. Provisions which, by their nature, must remain in effect beyond the termination of this License shall survive. + + 6.2. If You assert a patent infringement claim (excluding declaratory judgment actions) against Initial Developer or a Contributor (the Initial Developer or Contributor against whom You assert such claim is referred to as “Participant”) alleging that the Participant Software (meaning the Contributor Version where the Participant is a Contributor or the Original Software where the Participant is the Initial Developer) directly or indirectly infringes any patent, then any and all rights granted directly or indirectly to You by such Participant, the Initial Developer (if the Initial Developer is not the Participant) and all Contributors under Sections 2.1 and/or 2.2 of this License shall, upon 60 days notice from Participant terminate prospectively and automatically at the expiration of such 60 day notice period, unless if within such 60 day period You withdraw Your claim with respect to the Participant Software against such Participant either unilaterally or pursuant to a written agreement with Participant. + + 6.3. If You assert a patent infringement claim against Participant alleging that the Participant Software directly or indirectly infringes any patent where such claim is resolved (such as by license or settlement) prior to the initiation of patent infringement litigation, then the reasonable value of the licenses granted by such Participant under Sections 2.1 or 2.2 shall be taken into account in determining the amount or value of any payment or license. + + 6.4. In the event of termination under Sections 6.1 or 6.2 above, all end user licenses that have been validly granted by You or any distributor hereunder prior to termination (excluding licenses granted to You by any distributor) shall survive termination. + +7. LIMITATION OF LIABILITY. + +UNDER NO CIRCUMSTANCES AND UNDER NO LEGAL THEORY, WHETHER TORT (INCLUDING NEGLIGENCE), CONTRACT, OR OTHERWISE, SHALL YOU, THE INITIAL DEVELOPER, ANY OTHER CONTRIBUTOR, OR ANY DISTRIBUTOR OF COVERED SOFTWARE, OR ANY SUPPLIER OF ANY OF SUCH PARTIES, BE LIABLE TO ANY PERSON FOR ANY INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES OF ANY CHARACTER INCLUDING, WITHOUT LIMITATION, DAMAGES FOR LOSS OF GOODWILL, WORK STOPPAGE, COMPUTER FAILURE OR MALFUNCTION, OR ANY AND ALL OTHER COMMERCIAL DAMAGES OR LOSSES, EVEN IF SUCH PARTY SHALL HAVE BEEN INFORMED OF THE POSSIBILITY OF SUCH DAMAGES. THIS LIMITATION OF LIABILITY SHALL NOT APPLY TO LIABILITY FOR DEATH OR PERSONAL INJURY RESULTING FROM SUCH PARTY’S NEGLIGENCE TO THE EXTENT APPLICABLE LAW PROHIBITS SUCH LIMITATION. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION OR LIMITATION OF INCIDENTAL OR CONSEQUENTIAL DAMAGES, SO THIS EXCLUSION AND LIMITATION MAY NOT APPLY TO YOU. +8. U.S. GOVERNMENT END USERS. + +The Covered Software is a “commercial item,” as that term is defined in 48 C.F.R. 2.101 (Oct. 1995), consisting of “commercial computer software” (as that term is defined at 48 C.F.R. § 252.227-7014(a)(1)) and “commercial computer software documentation” as such terms are used in 48 C.F.R. 12.212 (Sept. 1995). Consistent with 48 C.F.R. 12.212 and 48 C.F.R. 227.7202-1 through 227.7202-4 (June 1995), all U.S. Government End Users acquire Covered Software with only those rights set forth herein. This U.S. Government Rights clause is in lieu of, and supersedes, any other FAR, DFAR, or other clause or provision that addresses Government rights in computer software under this License. +9. MISCELLANEOUS. + +This License represents the complete agreement concerning subject matter hereof. If any provision of this License is held to be unenforceable, such provision shall be reformed only to the extent necessary to make it enforceable. This License shall be governed by the law of the jurisdiction specified in a notice contained within the Original Software (except to the extent applicable law, if any, provides otherwise), excluding such jurisdiction’s conflict-of-law provisions. Any litigation relating to this License shall be subject to the jurisdiction of the courts located in the jurisdiction and venue specified in a notice contained within the Original Software, with the losing party responsible for costs, including, without limitation, court costs and reasonable attorneys’ fees and expenses. The application of the United Nations Convention on Contracts for the International Sale of Goods is expressly excluded. Any law or regulation which provides that the language of a contract shall be construed against the drafter shall not apply to this License. You agree that You alone are responsible for compliance with the United States export administration regulations (and the export control laws and regulation of any other countries) when You use, distribute or otherwise make available any Covered Software. +10. RESPONSIBILITY FOR CLAIMS. + +As between Initial Developer and the Contributors, each party is responsible for claims and damages arising, directly or indirectly, out of its utilization of rights under this License and You agree to work with Initial Developer and Contributors to distribute such responsibility on an equitable basis. Nothing herein is intended or shall be deemed to constitute any admission of liability. +NOTICE PURSUANT TO SECTION 9 OF THE COMMON DEVELOPMENT AND DISTRIBUTION LICENSE (CDDL) + +The code released under the CDDL shall be governed by the laws of the State of California (excluding conflict-of-law provisions). Any litigation relating to this License shall be subject to the jurisdiction of the Federal Courts of the Northern District of California and the state courts of the State of California, with venue lying in Santa Clara County, California. + +----- + +The following software may be included in this product: JLine. This software contains the following license and notice below: + +/* + * Copyright (c) 2002-2007, Marc Prud'hommeaux. All rights reserved. + * + * This software is distributable under the BSD license. See the terms of the + * BSD license in the documentation provided with this software. + */ + +----- + +The following software may be included in this product: Joni. This software contains the following license and notice below: + +/* +* Permission is hereby granted, free of charge, to any person obtaining a copy of +* this software and associated documentation files (the "Software"), to deal in +* the Software without restriction, including without limitation the rights to +* use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies +* of the Software, and to permit persons to whom the Software is furnished to do +* so, subject to the following conditions: +* +* The above copyright notice and this permission notice shall be included in all +* copies or substantial portions of the Software. +* +* THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +* IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +* FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +* AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +* LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +* OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +* SOFTWARE. +*/ + +----- + +The following software may be included in this product: leveldb. This software contains the following license and notice below: + +Copyright (c) 2011 The LevelDB Authors. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + +* Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. +* Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. +* Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +----- + +The following software may be included in this product: logback. The source code is available at http://logback.qos.ch/download.html. You may also request a copy of the source code by sending a request to opensource@fb.com. This software contains the following license and notice below: + +Eclipse Public License - v 1.0 + +THE ACCOMPANYING PROGRAM IS PROVIDED UNDER THE TERMS OF THIS ECLIPSE PUBLIC LICENSE ("AGREEMENT"). ANY USE, REPRODUCTION OR DISTRIBUTION OF THE PROGRAM CONSTITUTES RECIPIENT'S ACCEPTANCE OF THIS AGREEMENT. + +1. DEFINITIONS + +"Contribution" means: + +a) in the case of the initial Contributor, the initial code and documentation distributed under this Agreement, and + +b) in the case of each subsequent Contributor: + +i) changes to the Program, and + +ii) additions to the Program; + +where such changes and/or additions to the Program originate from and are distributed by that particular Contributor. A Contribution 'originates' from a Contributor if it was added to the Program by such Contributor itself or anyone acting on such Contributor's behalf. Contributions do not include additions to the Program which: (i) are separate modules of software distributed in conjunction with the Program under their own license agreement, and (ii) are not derivative works of the Program. + +"Contributor" means any person or entity that distributes the Program. + +"Licensed Patents" mean patent claims licensable by a Contributor which are necessarily infringed by the use or sale of its Contribution alone or when combined with the Program. + +"Program" means the Contributions distributed in accordance with this Agreement. + +"Recipient" means anyone who receives the Program under this Agreement, including all Contributors. + +2. GRANT OF RIGHTS + +a) Subject to the terms of this Agreement, each Contributor hereby grants Recipient a non-exclusive, worldwide, royalty-free copyright license to reproduce, prepare derivative works of, publicly display, publicly perform, distribute and sublicense the Contribution of such Contributor, if any, and such derivative works, in source code and object code form. + +b) Subject to the terms of this Agreement, each Contributor hereby grants Recipient a non-exclusive, worldwide, royalty-free patent license under Licensed Patents to make, use, sell, offer to sell, import and otherwise transfer the Contribution of such Contributor, if any, in source code and object code form. This patent license shall apply to the combination of the Contribution and the Program if, at the time the Contribution is added by the Contributor, such addition of the Contribution causes such combination to be covered by the Licensed Patents. The patent license shall not apply to any other combinations which include the Contribution. No hardware per se is licensed hereunder. + +c) Recipient understands that although each Contributor grants the licenses to its Contributions set forth herein, no assurances are provided by any Contributor that the Program does not infringe the patent or other intellectual property rights of any other entity. Each Contributor disclaims any liability to Recipient for claims brought by any other entity based on infringement of intellectual property rights or otherwise. As a condition to exercising the rights and licenses granted hereunder, each Recipient hereby assumes sole responsibility to secure any other intellectual property rights needed, if any. For example, if a third party patent license is required to allow Recipient to distribute the Program, it is Recipient's responsibility to acquire that license before distributing the Program. + +d) Each Contributor represents that to its knowledge it has sufficient copyright rights in its Contribution, if any, to grant the copyright license set forth in this Agreement. + +3. REQUIREMENTS + +A Contributor may choose to distribute the Program in object code form under its own license agreement, provided that: + +a) it complies with the terms and conditions of this Agreement; and + +b) its license agreement: + +i) effectively disclaims on behalf of all Contributors all warranties and conditions, express and implied, including warranties or conditions of title and non-infringement, and implied warranties or conditions of merchantability and fitness for a particular purpose; + +ii) effectively excludes on behalf of all Contributors all liability for damages, including direct, indirect, special, incidental and consequential damages, such as lost profits; + +iii) states that any provisions which differ from this Agreement are offered by that Contributor alone and not by any other party; and + +iv) states that source code for the Program is available from such Contributor, and informs licensees how to obtain it in a reasonable manner on or through a medium customarily used for software exchange. + +When the Program is made available in source code form: + +a) it must be made available under this Agreement; and + +b) a copy of this Agreement must be included with each copy of the Program. + +Contributors may not remove or alter any copyright notices contained within the Program. + +Each Contributor must identify itself as the originator of its Contribution, if any, in a manner that reasonably allows subsequent Recipients to identify the originator of the Contribution. + +4. COMMERCIAL DISTRIBUTION + +Commercial distributors of software may accept certain responsibilities with respect to end users, business partners and the like. While this license is intended to facilitate the commercial use of the Program, the Contributor who includes the Program in a commercial product offering should do so in a manner which does not create potential liability for other Contributors. Therefore, if a Contributor includes the Program in a commercial product offering, such Contributor ("Commercial Contributor") hereby agrees to defend and indemnify every other Contributor ("Indemnified Contributor") against any losses, damages and costs (collectively "Losses") arising from claims, lawsuits and other legal actions brought by a third party against the Indemnified Contributor to the extent caused by the acts or omissions of such Commercial Contributor in connection with its distribution of the Program in a commercial product offering. The obligations in this section do not apply to any claims or Losses relating to any actual or alleged intellectual property infringement. In order to qualify, an Indemnified Contributor must: a) promptly notify the Commercial Contributor in writing of such claim, and b) allow the Commercial Contributor to control, and cooperate with the Commercial Contributor in, the defense and any related settlement negotiations. The Indemnified Contributor may participate in any such claim at its own expense. + +For example, a Contributor might include the Program in a commercial product offering, Product X. That Contributor is then a Commercial Contributor. If that Commercial Contributor then makes performance claims, or offers warranties related to Product X, those performance claims and warranties are such Commercial Contributor's responsibility alone. Under this section, the Commercial Contributor would have to defend claims against the other Contributors related to those performance claims and warranties, and if a court requires any other Contributor to pay any damages as a result, the Commercial Contributor must pay those damages. + +5. NO WARRANTY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, THE PROGRAM IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, EITHER EXPRESS OR IMPLIED INCLUDING, WITHOUT LIMITATION, ANY WARRANTIES OR CONDITIONS OF TITLE, NON-INFRINGEMENT, MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE. Each Recipient is solely responsible for determining the appropriateness of using and distributing the Program and assumes all risks associated with its exercise of rights under this Agreement , including but not limited to the risks and costs of program errors, compliance with applicable laws, damage to or loss of data, programs or equipment, and unavailability or interruption of operations. + +6. DISCLAIMER OF LIABILITY + +EXCEPT AS EXPRESSLY SET FORTH IN THIS AGREEMENT, NEITHER RECIPIENT NOR ANY CONTRIBUTORS SHALL HAVE ANY LIABILITY FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING WITHOUT LIMITATION LOST PROFITS), HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OR DISTRIBUTION OF THE PROGRAM OR THE EXERCISE OF ANY RIGHTS GRANTED HEREUNDER, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGES. + +7. GENERAL + +If any provision of this Agreement is invalid or unenforceable under applicable law, it shall not affect the validity or enforceability of the remainder of the terms of this Agreement, and without further action by the parties hereto, such provision shall be reformed to the minimum extent necessary to make such provision valid and enforceable. + +If Recipient institutes patent litigation against any entity (including a cross-claim or counterclaim in a lawsuit) alleging that the Program itself (excluding combinations of the Program with other software or hardware) infringes such Recipient's patent(s), then such Recipient's rights granted under Section 2(b) shall terminate as of the date such litigation is filed. + +All Recipient's rights under this Agreement shall terminate if it fails to comply with any of the material terms or conditions of this Agreement and does not cure such failure in a reasonable period of time after becoming aware of such noncompliance. If all Recipient's rights under this Agreement terminate, Recipient agrees to cease use and distribution of the Program as soon as reasonably practicable. However, Recipient's obligations under this Agreement and any licenses granted by Recipient relating to the Program shall continue and survive. + +Everyone is permitted to copy and distribute copies of this Agreement, but in order to avoid inconsistency the Agreement is copyrighted and may only be modified in the following manner. The Agreement Steward reserves the right to publish new versions (including revisions) of this Agreement from time to time. No one other than the Agreement Steward has the right to modify this Agreement. The Eclipse Foundation is the initial Agreement Steward. The Eclipse Foundation may assign the responsibility to serve as the Agreement Steward to a suitable separate entity. Each new version of the Agreement will be given a distinguishing version number. The Program (including Contributions) may always be distributed subject to the version of the Agreement under which it was received. In addition, after a new version of the Agreement is published, Contributor may elect to distribute the Program (including its Contributions) under the new version. Except as expressly stated in Sections 2(a) and 2(b) above, Recipient receives no rights or licenses to the intellectual property of any Contributor under this Agreement, whether expressly, by implication, estoppel or otherwise. All rights in the Program not expressly granted under this Agreement are reserved. + +This Agreement is governed by the laws of the State of New York and the intellectual property laws of the United States of America. No party to this Agreement will bring a legal action under this Agreement more than one year after the cause of action arose. Each party waives its rights to a jury trial in any resulting litigation. + +----- + +The following software may be included in this product: protobuf. This software contains the following license and notice below: + +Copyright 2008, Google Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + +* Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. +* Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. +* Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +Code generated by the Protocol Buffer compiler is owned by the owner +of the input file used when generating it. This code is not +standalone and requires a support library to be linked with it. This +support library is itself covered by the above license. + +----- + +The following software may be included in this product: slf4j. This software contains the following license and notice below: + +Copyright (c) 2004-2013 QOS.ch +All rights reserved. + +Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. diff --git a/presto-twitter-server/README.txt b/presto-twitter-server/README.txt new file mode 100644 index 000000000000..46017e937c7b --- /dev/null +++ b/presto-twitter-server/README.txt @@ -0,0 +1,5 @@ +Presto is a distributed SQL query engine. + +Please see the website for installation instructions: + +https://prestodb.io/ diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml new file mode 100644 index 000000000000..cb2cd43fc47c --- /dev/null +++ b/presto-twitter-server/pom.xml @@ -0,0 +1,26 @@ + + + 4.0.0 + + + com.facebook.presto + presto-root + 0.181-tw-0.39 + + + presto-twitter-server + presto-twitter-server + provisio + + + ${project.parent.basedir} + true + true + true + true + + + com.facebook.presto.server.PrestoServer + ${project.artifactId} + + diff --git a/presto-twitter-server/src/main/provisio/presto.xml b/presto-twitter-server/src/main/provisio/presto.xml new file mode 100644 index 000000000000..64124eefd08b --- /dev/null +++ b/presto-twitter-server/src/main/provisio/presto.xml @@ -0,0 +1,64 @@ + + + + + + + + NOTICE + README.txt + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/presto-twitter-server/src/main/provisio/twitter.xml b/presto-twitter-server/src/main/provisio/twitter.xml new file mode 100644 index 000000000000..9570c58fcfb7 --- /dev/null +++ b/presto-twitter-server/src/main/provisio/twitter.xml @@ -0,0 +1,16 @@ + + + + + + + + + + + + + + + + From 4d7c92c6a75261439021f828692e346cac9f345a Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Mon, 16 Oct 2017 18:59:46 -0700 Subject: [PATCH 225/331] Point the thrift id into NON_EXISTED_THRIFT_ID if the id resolver failed --- .../thrift/HiveThriftFieldIdResolver.java | 2 + .../hive/thrift/ThriftHiveRecordCursor.java | 46 +++++++++---------- 2 files changed, 23 insertions(+), 25 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java index 67a3088afb30..077a409fd2a2 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/HiveThriftFieldIdResolver.java @@ -72,6 +72,7 @@ public HiveThriftFieldIdResolver(JsonNode root) this.root = root; } + @Override public short getThriftId(int hiveIndex) { if (root == null) { @@ -97,6 +98,7 @@ public short getThriftId(int hiveIndex) } } + @Override public ThriftFieldIdResolver getNestedResolver(int hiveIndex) { if (root == null) { diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java index 122ced137231..20927497fdc8 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java @@ -81,6 +81,7 @@ class ThriftHiveRecordCursor implements RecordCursor { private static final Logger log = Logger.get(ThriftHiveRecordCursor.class); + private static final short NON_EXISTED_THRIFT_ID = (short) -1; private final RecordReader recordReader; private final K key; private final V value; @@ -165,7 +166,12 @@ public ThriftHiveRecordCursor( types[i] = typeManager.getType(column.getTypeSignature()); hiveTypes[i] = column.getHiveType(); hiveIndexs[i] = column.getHiveColumnIndex(); - thriftIds[i] = thriftFieldIdResolver.getThriftId(hiveIndexs[i]); + try { + thriftIds[i] = thriftFieldIdResolver.getThriftId(hiveIndexs[i]); + } + catch (PrestoException e) { + thriftIds[i] = NON_EXISTED_THRIFT_ID; + } } } @@ -546,6 +552,10 @@ private static Block getBlockObject(Type type, ThriftFieldIdResolver resolver, O private static Block serializeObject(Type type, ThriftFieldIdResolver resolver, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) { + if (object == null) { + requireNonNull(builder, "parent builder is null").appendNull(); + return null; + } if (!isStructuralType(type)) { serializePrimitive(type, resolver, builder, object, hiveStorageTimeZone); return null; @@ -564,12 +574,7 @@ else if (isRowType(type)) { private static Block serializeList(Type type, ThriftFieldIdResolver resolver, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) { - List list = (List) object; - if (list == null) { - requireNonNull(builder, "parent builder is null").appendNull(); - return null; - } - + List list = (List) requireNonNull(object, "object is null"); List typeParameters = type.getTypeParameters(); checkArgument(typeParameters.size() == 1, "list must have exactly 1 type parameter"); Type elementType = typeParameters.get(0); @@ -598,12 +603,7 @@ private static Block serializeList(Type type, ThriftFieldIdResolver resolver, Bl private static Block serializeMap(Type type, ThriftFieldIdResolver resolver, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) { - Map map = (Map) object; - if (map == null) { - requireNonNull(builder, "parent builder is null").appendNull(); - return null; - } - + Map map = (Map) requireNonNull(object, "object is null"); List typeParameters = type.getTypeParameters(); checkArgument(typeParameters.size() == 2, "map must have exactly 2 type parameter"); Type keyType = typeParameters.get(0); @@ -638,13 +638,8 @@ private static Block serializeMap(Type type, ThriftFieldIdResolver resolver, Blo private static Block serializeStruct(Type type, ThriftFieldIdResolver resolver, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) { - if (object == null) { - requireNonNull(builder, "parent builder is null").appendNull(); - return null; - } - + ThriftGenericRow structData = (ThriftGenericRow) requireNonNull(object, "object is null"); List typeParameters = type.getTypeParameters(); - ThriftGenericRow structData = (ThriftGenericRow) object; BlockBuilder currentBuilder; if (builder != null) { currentBuilder = builder.beginBlockEntry(); @@ -655,7 +650,12 @@ private static Block serializeStruct(Type type, ThriftFieldIdResolver resolver, for (int i = 0; i < typeParameters.size(); i++) { Object fieldValue = structData.getFieldValueForThriftId(resolver.getThriftId(i)); - serializeObject(typeParameters.get(i), resolver.getNestedResolver(i), currentBuilder, fieldValue, hiveStorageTimeZone); + if (fieldValue == null) { + currentBuilder.appendNull(); + } + else { + serializeObject(typeParameters.get(i), resolver.getNestedResolver(i), currentBuilder, fieldValue, hiveStorageTimeZone); + } } if (builder != null) { @@ -671,11 +671,7 @@ private static Block serializeStruct(Type type, ThriftFieldIdResolver resolver, private static void serializePrimitive(Type type, ThriftFieldIdResolver resolver, BlockBuilder builder, Object object, DateTimeZone hiveStorageTimeZone) { requireNonNull(builder, "parent builder is null"); - - if (object == null) { - builder.appendNull(); - return; - } + requireNonNull(object, "object is null"); if (BOOLEAN.equals(type)) { BOOLEAN.writeBoolean(builder, (Boolean) object); From 74875d7b17aa81e2a9edc767815b4ce54effe94e Mon Sep 17 00:00:00 2001 From: David Phillips Date: Tue, 17 Oct 2017 13:30:59 -0700 Subject: [PATCH 226/331] Use default Java version for Travis CI --- .travis.yml | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/.travis.yml b/.travis.yml index 73a8ea4bafea..6a2bb15ebd2d 100644 --- a/.travis.yml +++ b/.travis.yml @@ -23,11 +23,7 @@ env: sudo: required dist: trusty -group: deprecated-2017Q2 -addons: - apt: - packages: - - oracle-java8-installer +group: deprecated-2017Q3 cache: directories: From 43588a873e5912782a937ead6c4c383a45067d11 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 18 Oct 2017 13:56:33 -0700 Subject: [PATCH 227/331] Move back to oss version --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 51 files changed, 53 insertions(+), 53 deletions(-) diff --git a/pom.xml b/pom.xml index 43c4c5ed0d5a..7d62cbbf5a2d 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.181-tw-0.39 + 0.181 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index caba67b10469..1446f0ecaec5 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 3b453d7ed602..f1d2ef401da8 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index d058d420dfc9..680ae3708c65 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 253be3f4f828..6f79af2dd5d0 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 84684425eaad..6ee2347ecf8e 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index c1378e82548a..b89c60b5aa2d 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.181-tw-0.39 + 0.181 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 36738d7a5a60..2dd093e29d26 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index f981a9b97747..f1fd62ef7814 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 4915d365c9b5..3085a1e8a666 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 6945381febf5..0b03d9918c44 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index e3bc71d4bd61..c8905d75b87b 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 255f0f4da066..6529190bc3d9 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index ebadda9a1234..5f3c2a90297c 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 85c92bbbdf36..6fc8e13966cc 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-example-http diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 0a1f9f5786a5..6615ffe5325f 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index beeb70d05a48..730a96e8b62d 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index ff207598132a..de7c493b5d56 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index b64533843c52..f0a62b340c5b 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 7178c908f307..c86c46262a38 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index d2d51bd1c746..9e5f90a34ef9 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 68364a504dcc..58ef713653dd 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 690581f23e91..eb6e5d174153 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-main diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 3cc1c4d7aa16..93e05242b469 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 50145f28d1a6..9506b29cd009 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index f3ce0df6c2a7..0ebc1ca30716 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 181d62aa9ed8..fedb182eaf9e 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 7829ce3ad967..2fb13d5fee89 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 314f4138d03d..afe831d4aa33 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 40928cb0c66c..44470f8a8714 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 25bad2300d3d..eaeff5a5b52a 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 43474d17c92c..f6d724a735cf 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.181-tw-0.39 + 0.181 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index d8937574a666..6cf07e83e992 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index ae813f72db8b..3a5c990b2ad1 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 08a272642d87..34e08a02c1de 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 8f7fb4e8778a..72792a641114 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-redis diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index d6b082348cf9..0bfd1a9c5bcc 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 7bbd20ee6070..4dd80298ada9 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index b18a1b4f11bf..3c0dc98b52aa 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 0d487f302f3a..b536e7977c59 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index bb06e51737ad..de9ed48a4945 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.181-tw-0.39 + 0.181 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index b3f28afde8aa..8fd5091f3c5b 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 48ec4ff2f48a..296043bb8e0d 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index dde9938eaee7..29a0bdbe0917 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.181-tw-0.39 + 0.181 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index 676435bd1448..ede302110fd1 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 8e64b53209f3..d42b474edde3 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index 8b81be71a997..b46e7ab9627d 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-thrift-testing-server diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 43e6a4a4d26e..796bd8c6837b 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-tpch diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index cb2cd43fc47c..ef7d49ff2468 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 3303ca264fd4..aec30ce9fac1 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 145708cb8d7e..171f4ad95e04 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181-tw-0.39 + 0.181 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.181-tw-0.39 + 0.181 provided From ffa86a612daef00f6a248abeda8bc9bd560949b9 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 18 Oct 2017 14:42:07 -0700 Subject: [PATCH 228/331] Adapt thrift reader --- .../hive/thrift/ThriftHiveRecordCursor.java | 15 ++++----------- 1 file changed, 4 insertions(+), 11 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java index 122ced137231..32513bf928ad 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java @@ -20,7 +20,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.DecimalType; import com.facebook.presto.spi.type.Decimals; import com.facebook.presto.spi.type.Type; @@ -57,7 +56,7 @@ import static com.facebook.presto.spi.type.BigintType.BIGINT; import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; import static com.facebook.presto.spi.type.Chars.isCharType; -import static com.facebook.presto.spi.type.Chars.trimSpacesAndTruncateToLength; +import static com.facebook.presto.spi.type.Chars.truncateToLengthAndTrimSpaces; import static com.facebook.presto.spi.type.DateType.DATE; import static com.facebook.presto.spi.type.Decimals.rescale; import static com.facebook.presto.spi.type.DoubleType.DOUBLE; @@ -169,12 +168,6 @@ public ThriftHiveRecordCursor( } } - @Override - public long getTotalBytes() - { - return totalBytes; - } - @Override public long getCompletedBytes() { @@ -396,7 +389,7 @@ else if (value instanceof Integer) { sliceValue = truncateToLength(sliceValue, type); } if (isCharType(type)) { - sliceValue = trimSpacesAndTruncateToLength(sliceValue, type); + sliceValue = truncateToLengthAndTrimSpaces(sliceValue, type); } return sliceValue; @@ -615,7 +608,7 @@ private static Block serializeMap(Type type, ThriftFieldIdResolver resolver, Blo currentBuilder = builder.beginBlockEntry(); } else { - currentBuilder = new InterleavedBlockBuilder(typeParameters, new BlockBuilderStatus(), map.size()); + currentBuilder = type.createBlockBuilder(new BlockBuilderStatus(), map.size()); } for (Map.Entry entry : map.entrySet()) { @@ -650,7 +643,7 @@ private static Block serializeStruct(Type type, ThriftFieldIdResolver resolver, currentBuilder = builder.beginBlockEntry(); } else { - currentBuilder = new InterleavedBlockBuilder(typeParameters, new BlockBuilderStatus(), typeParameters.size()); + currentBuilder = type.createBlockBuilder(new BlockBuilderStatus(), typeParameters.size()); } for (int i = 0; i < typeParameters.size(); i++) { From 0ce96b4ba513637585dc0aeb87695c9f77f19022 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 18 Oct 2017 14:46:17 -0700 Subject: [PATCH 229/331] bump up version to 0.186-tw-0.40 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 55 files changed, 57 insertions(+), 57 deletions(-) diff --git a/pom.xml b/pom.xml index 1d566c6c0716..2597def7f024 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.186 + 0.186-tw-0.40 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 4e3f10105fbf..5e42dee28db8 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 3467144cdf78..011076004286 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index c0c0229e8235..54a51004bf1c 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index eb2ae8f710df..599b90776974 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index ee9c1dc5c338..f61eac6950e6 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index f02cdb406b7f..dfad3a307b59 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.186 + 0.186-tw-0.40 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index efc7cd49b351..4518855f718a 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index b0632e764c1b..ebdd94c220c3 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 95051e7de3bb..e65e6c921e7b 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index eab18e0e8713..7f3316877167 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 8d6952e9000e..d5e81432ade5 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index e7a3c7907413..d95b3829d95e 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 135c90900d6e..29d50e323a91 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 1c3d58e4d66c..d2eababf0733 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-example-http diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index 0356cf9bce69..e8d6c8a4e1cd 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 23772a96aef8..1879014e9600 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index fc7fa1160f60..c1f6c086c10a 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index bb5d61d20666..c98b6edb7c8b 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index a6c34ef6b3cd..49c01b85454c 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index d354a8e05d7b..e0c4b94faa78 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index 9e5f90a34ef9..20d6836ce41a 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.186-tw-0.40 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index e724bc0f8b8c..f0281298b9a6 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 0519e4ff8528..39db44d781a3 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index 5dcadc6658cc..9810d2cc6522 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.186 + 0.186-tw-0.40 presto-matching diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index d79357a77d6f..18c96fb391bb 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 2664dad8453a..510a66e8883c 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 38663898ec8d..e0039e9c7474 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 5a778ad18797..ad77789e5f35 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 274e49e5c628..35bc32ab5f61 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 915f4bf53447..f4b9dfc8bed5 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 36d6463a6d67..26828c1b7ea9 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index f5d66dfe9504..b3166c93019b 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index a8fb1b83161e..fbec4548445e 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.186 + 0.186-tw-0.40 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 0b2e8fc99995..bc5f3996d0f9 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index dfa72802b72c..a2144c466cf9 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index f64076dd5cb1..67994203e1e3 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 4f67bce04495..10f40b16c1fb 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index 666192caac6c..49a56619df60 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 96fe4ac23b5f..a1389a95f0df 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index fcc36f9cdc47..700b1f45ce49 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 22e522539805..53fa54923ea8 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 8faca4c35e5c..09de8fc4fc43 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 43f1cea09dec..19673a3844fa 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.186 + 0.186-tw-0.40 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 30e5c6b4468e..41614845a60e 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 40d7337dd76b..57beeba35d18 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index f7a7c013a432..81af859ca8e7 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.186 + 0.186-tw-0.40 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index ed6173555651..7d3964550c86 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 8b139fcecdac..0d0e5214948f 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index 71527cd70eeb..fbd6d458329c 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index 0bdcc0773e1b..5935f44da828 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index b62bc93cf3cd..aa3fa56a5f0a 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-tpch diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index ef7d49ff2468..ae79faf0f5af 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.181 + 0.186-tw-0.40 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 1de03c404aa7..c6d24b4e4340 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.186-tw-0.40 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 171f4ad95e04..34aa8ee06458 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.181 + 0.186-tw-0.40 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.181 + 0.186-tw-0.40 provided From 5572f0cd1d5e773c828f43c2e6ca748c292be19d Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 18 Oct 2017 18:50:43 -0700 Subject: [PATCH 230/331] correct code style --- pom.xml | 58 +++- presto-hive/pom.xml | 16 +- .../hive/BackgroundHiveSplitLoader.java | 4 +- .../presto/hive/HivePageSourceProvider.java | 3 +- .../PooledHiveMetastoreClientFactory.java | 26 +- .../hive/ZookeeperMetastoreMonitor.java | 3 +- .../hive/thrift/ThriftGeneralInputFormat.java | 3 +- .../twitter/hive/thrift/ThriftGenericRow.java | 43 ++- .../hive/util/PooledTTransportFactory.java | 31 +- .../twitter/hive/util/TTransportPool.java | 8 +- .../presto/hive/TestHiveFileFormats.java | 16 +- .../presto/kafka/KafkaConnectorFactory.java | 3 +- .../facebook/presto/kafka/KafkaRecordSet.java | 8 +- .../presto/kafka/KafkaSplitManager.java | 2 +- .../kafka/TestKafkaConnectorConfig.java | 3 +- .../presto/kafka/TestKafkaDistributed.java | 3 + .../kafka/TestKafkaIntegrationSmokeTest.java | 3 + .../presto/kafka/util/EmbeddedKafka.java | 6 +- .../presto/kafka/util/EmbeddedZookeeper.java | 6 +- .../presto/kafka/util/NumberPartitioner.java | 3 +- .../facebook/presto/kafka/util/TestUtils.java | 3 +- .../decoder/thrift/ThriftDecoderModule.java | 16 +- .../decoder/thrift/ThriftFieldDecoder.java | 9 +- .../decoder/thrift/ThriftGenericRow.java | 47 ++- .../QueryCompletedEventScriber.java | 144 ++++---- .../eventlistener/QueryStatsHelper.java | 318 +++++++++--------- .../eventlistener/TwitterEventListener.java | 33 +- .../TwitterEventListenerFactory.java | 23 +- .../TwitterEventListenerPlugin.java | 13 +- .../plugin/eventlistener/TwitterScriber.java | 84 ++--- 30 files changed, 507 insertions(+), 431 deletions(-) diff --git a/pom.xml b/pom.xml index 2597def7f024..d806b791f6ed 100644 --- a/pom.xml +++ b/pom.xml @@ -852,12 +852,58 @@ 1.1.1.7 + + org.apache.curator + curator-recipes + 4.0.0 + + + org.apache.zookeeper + zookeeper + + + + + + org.apache.curator + curator-framework + 4.0.0 + + + org.apache.zookeeper + zookeeper + + + + + + org.apache.curator + curator-client + 4.0.0 + + + org.apache.zookeeper + zookeeper + + + + + + org.apache.curator + curator-test + 4.0.0 + + + org.apache.zookeeper + zookeeper + + + + org.apache.zookeeper zookeeper - - - 3.5.1-alpha + 3.4.9 jline @@ -877,7 +923,7 @@ com.101tec zkclient - 0.8 + 0.10 log4j @@ -887,6 +933,10 @@ org.slf4j slf4j-log4j12 + + org.apache.zookeeper + zookeeper + diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index c1f6c086c10a..b24dbcbc524c 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -51,19 +51,22 @@ org.apache.curator curator-recipes - 3.3.0 org.apache.curator curator-framework - 3.3.0 org.apache.curator curator-client - 3.3.0 + + + + org.apache.zookeeper + zookeeper + runtime @@ -274,13 +277,6 @@ org.apache.curator curator-test - 3.3.0 - test - - - - org.apache.zookeeper - zookeeper test 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 b5de0efcf58f..abeb06c3f851 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 @@ -573,9 +573,7 @@ private Iterator createHiveSplitIterator( return new AbstractIterator() { private long chunkOffset = 0; - private LzoIndex index = isLzopCompressedFile(filePath) ? - LzoIndex.readIndex(hdfsEnvironment.getFileSystem(hdfsContext, getLzopIndexPath(filePath)), filePath) : - null; + private LzoIndex index = isLzopCompressedFile(filePath) ? LzoIndex.readIndex(hdfsEnvironment.getFileSystem(hdfsContext, getLzopIndexPath(filePath)), filePath) : null; @Override protected HiveSplit computeNext() 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 271b3e630fbf..2256553f6039 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 @@ -86,8 +86,7 @@ public ConnectorPageSource createPageSource(ConnectorTransactionHandle transacti UserGroupInformation ugi = UgiUtils.getUgi(session.getUser()); try { return ugi.doAs((PrivilegedExceptionAction) () -> - doCreatePageSource(session, split, columns) - ); + doCreatePageSource(session, split, columns)); } catch (IOException | InterruptedException e) { throw new RuntimeException("Could not runAs " + session.getUser(), e); diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java index c67e921dfad8..8bbb11972338 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java @@ -38,9 +38,13 @@ public class PooledHiveMetastoreClientFactory private final HiveMetastoreAuthentication metastoreAuthentication; private final TTransportPool transportPool; - public PooledHiveMetastoreClientFactory(@Nullable HostAndPort socksProxy, Duration timeout, - HiveMetastoreAuthentication metastoreAuthentication, - int maxTransport, long idleTimeout, long transportEvictInterval, int evictNumTests) + public PooledHiveMetastoreClientFactory(@Nullable HostAndPort socksProxy, + Duration timeout, + HiveMetastoreAuthentication metastoreAuthentication, + int maxTransport, + long idleTimeout, + long transportEvictInterval, + int evictNumTests) { this.socksProxy = socksProxy; this.timeoutMillis = toIntExact(timeout.toMillis()); @@ -56,16 +60,16 @@ public PooledHiveMetastoreClientFactory(@Nullable HostAndPort socksProxy, Durati @Inject public PooledHiveMetastoreClientFactory(HiveClientConfig config, - ZookeeperServersetMetastoreConfig zkConfig, - HiveMetastoreAuthentication metastoreAuthentication) + ZookeeperServersetMetastoreConfig zkConfig, + HiveMetastoreAuthentication metastoreAuthentication) { this(config.getMetastoreSocksProxy(), - config.getMetastoreTimeout(), - metastoreAuthentication, - zkConfig.getMaxTransport(), - zkConfig.getTransportIdleTimeout(), - zkConfig.getTransportEvictInterval(), - zkConfig.getTransportEvictNumTests()); + config.getMetastoreTimeout(), + metastoreAuthentication, + zkConfig.getMaxTransport(), + zkConfig.getTransportIdleTimeout(), + zkConfig.getTransportEvictInterval(), + zkConfig.getTransportEvictNumTests()); } public HiveMetastoreClient create(String host, int port) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperMetastoreMonitor.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperMetastoreMonitor.java index e1d0f2011468..f340abf47334 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperMetastoreMonitor.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperMetastoreMonitor.java @@ -33,7 +33,8 @@ import java.util.concurrent.ConcurrentMap; import java.util.stream.Collectors; -public class ZookeeperMetastoreMonitor implements PathChildrenCacheListener +public class ZookeeperMetastoreMonitor + implements PathChildrenCacheListener { public static final Logger log = Logger.get(ZookeeperMetastoreMonitor.class); private CuratorFramework client; diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java index 181a463dc25a..fea6a8d76875 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java @@ -39,7 +39,8 @@ * Remove the class once #481 is included in a release */ @SuppressWarnings("deprecation") -public class ThriftGeneralInputFormat extends DeprecatedFileInputFormatWrapper +public class ThriftGeneralInputFormat + extends DeprecatedFileInputFormatWrapper { public ThriftGeneralInputFormat() { diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java index 7f157ef95523..c319bba2d375 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java @@ -37,7 +37,8 @@ import java.util.Map; import java.util.Set; -public class ThriftGenericRow implements TBase +public class ThriftGenericRow + implements TBase { private static final Logger log = Logger.get(ThriftGenericRow.class); private final Map values = new HashMap<>(); @@ -45,14 +46,17 @@ public class ThriftGenericRow implements TBase values) { this.values.putAll(values); } - public class Fields implements TFieldIdEnum + public class Fields + implements TFieldIdEnum { private final short thriftId; private final String fieldName; @@ -74,7 +78,8 @@ public String getFieldName() } } - public void read(TProtocol iprot) throws TException + public void read(TProtocol iprot) + throws TException { TTransport trans = iprot.getTransport(); buf = trans.getBuffer(); @@ -83,12 +88,14 @@ public void read(TProtocol iprot) throws TException len = trans.getBufferPosition() - off; } - public void parse() throws TException + public void parse() + throws TException { parse(null); } - public void parse(short[] thriftIds) throws TException + public void parse(short[] thriftIds) + throws TException { Set idSet = thriftIds == null ? null : new HashSet(Arrays.asList(ArrayUtils.toObject(thriftIds))); TMemoryInputTransport trans = new TMemoryInputTransport(buf, off, len); @@ -111,7 +118,8 @@ public void parse(short[] thriftIds) throws TException iprot.readStructEnd(); } - private Object readElem(TProtocol iprot, byte type) throws TException + private Object readElem(TProtocol iprot, byte type) + throws TException { switch (type) { case TType.BOOL: @@ -143,7 +151,8 @@ private Object readElem(TProtocol iprot, byte type) throws TException } } - private Object readStruct(TProtocol iprot) throws TException + private Object readStruct(TProtocol iprot) + throws TException { ThriftGenericRow elem = new ThriftGenericRow(); elem.read(iprot); @@ -151,33 +160,36 @@ private Object readStruct(TProtocol iprot) throws TException return elem; } - private Object readList(TProtocol iprot) throws TException + private Object readList(TProtocol iprot) + throws TException { TList ilist = iprot.readListBegin(); List listValue = new ArrayList<>(); - for (int i = 0; i < ilist.size; ++i) { + for (int i = 0; i < ilist.size; i++) { listValue.add(readElem(iprot, ilist.elemType)); } iprot.readListEnd(); return listValue; } - private Object readSet(TProtocol iprot) throws TException + private Object readSet(TProtocol iprot) + throws TException { TSet iset = iprot.readSetBegin(); List setValue = new ArrayList<>(); - for (int i = 0; i < iset.size; ++i) { + for (int i = 0; i < iset.size; i++) { setValue.add(readElem(iprot, iset.elemType)); } iprot.readSetEnd(); return setValue; } - private Object readMap(TProtocol iprot) throws TException + private Object readMap(TProtocol iprot) + throws TException { TMap imap = iprot.readMapBegin(); Map mapValue = new HashMap<>(); - for (int i = 0; i < imap.size; ++i) { + for (int i = 0; i < imap.size; i++) { mapValue.put(readElem(iprot, imap.keyType), readElem(iprot, imap.valueType)); } iprot.readMapEnd(); @@ -216,7 +228,8 @@ public void setFieldValue(Fields field, Object value) values.put(field.getThriftFieldId(), value); } - public void write(TProtocol oprot) throws TException + public void write(TProtocol oprot) + throws TException { throw new UnsupportedOperationException("ThriftGenericRow.write is not supported."); } diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java index 68efa43131cb..554435e12923 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java @@ -34,7 +34,7 @@ import static java.util.Objects.requireNonNull; public class PooledTTransportFactory - extends BasePooledObjectFactory + extends BasePooledObjectFactory { private final TTransportPool pool; private final String host; @@ -43,9 +43,12 @@ public class PooledTTransportFactory private final int timeoutMillis; private final HiveMetastoreAuthentication metastoreAuthentication; - public PooledTTransportFactory(TTransportPool pool, String host, int port, - @Nullable HostAndPort socksProxy, int timeoutMillis, - HiveMetastoreAuthentication metastoreAuthentication) + public PooledTTransportFactory(TTransportPool pool, + String host, + int port, + @Nullable HostAndPort socksProxy, + int timeoutMillis, + HiveMetastoreAuthentication metastoreAuthentication) { this.pool = requireNonNull(pool, "pool is null"); this.host = requireNonNull(host, "host is null"); @@ -57,7 +60,7 @@ public PooledTTransportFactory(TTransportPool pool, String host, int port, @Override public void activateObject(PooledObject pooledObject) - throws Exception + throws Exception { pooledObject.getObject().flush(); } @@ -76,7 +79,7 @@ public boolean validateObject(PooledObject pooledObject) @Override public TTransport create() - throws Exception + throws Exception { TTransport transport; if (socksProxy == null) { @@ -142,7 +145,7 @@ public void passivateObject(PooledObject pooledObject) } private static class PooledTTransport - extends TTransport + extends TTransport { private final String remote; private final TTransportPool pool; @@ -161,7 +164,7 @@ public TTransport getTTransport() } public boolean isReachable(int timeoutMillis) - throws ClassCastException, IOException + throws ClassCastException, IOException { return ((TSocket) transport).getSocket().getInetAddress().isReachable(timeoutMillis); } @@ -215,42 +218,42 @@ public void consumeBuffer(int len) @Override public void open() - throws TTransportException + throws TTransportException { transport.open(); } @Override public int readAll(byte[] bytes, int off, int len) - throws TTransportException + throws TTransportException { return transport.readAll(bytes, off, len); } @Override public int read(byte[] bytes, int off, int len) - throws TTransportException + throws TTransportException { return transport.read(bytes, off, len); } @Override public void write(byte[] bytes) - throws TTransportException + throws TTransportException { transport.write(bytes); } @Override public void write(byte[] bytes, int off, int len) - throws TTransportException + throws TTransportException { transport.write(bytes, off, len); } @Override public void flush() - throws TTransportException + throws TTransportException { transport.flush(); } diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java index 0e34f6756f1c..7d2a3b5af23b 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/TTransportPool.java @@ -39,14 +39,14 @@ protected synchronized void add(String remote, PooledObjectFactory transportFact } protected TTransport get(String remote, PooledObjectFactory transportFactory) - throws Exception + throws Exception { add(remote, transportFactory); return get(remote); } protected TTransport get(String remote) - throws Exception + throws Exception { ObjectPool pool = pools.get(remote); if (pool == null) { @@ -56,13 +56,13 @@ protected TTransport get(String remote) } public TTransport borrowObject(String host, int port, PooledObjectFactory transportFactory) - throws Exception + throws Exception { return get(HostAndPort.fromParts(host, port).toString(), transportFactory); } public TTransport borrowObject(String host, int port) - throws Exception + throws Exception { return get(HostAndPort.fromParts(host, port).toString()); } 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 29b8d65d88ea..b01b33259b70 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 @@ -515,30 +515,20 @@ public void testLZOThrift(int rowCount) ImmutableList.of( getStandardStructObjectInspector( ImmutableList.of("first_name", "last_name"), - ImmutableList.of(javaStringObjectInspector, javaStringObjectInspector) - ), + ImmutableList.of(javaStringObjectInspector, javaStringObjectInspector)), javaIntObjectInspector, javaStringObjectInspector, getStandardListObjectInspector( getStandardStructObjectInspector( ImmutableList.of("number", "type"), - ImmutableList.of(javaStringObjectInspector, javaStringObjectInspector) - ) - ) - ) - ) - ), + ImmutableList.of(javaStringObjectInspector, javaStringObjectInspector)))))), null, arrayBlockOf(personType, rowBlockOf(ImmutableList.of(nameType, INTEGER, createUnboundedVarcharType(), new ArrayType(phoneType)), rowBlockOf(ImmutableList.of(createUnboundedVarcharType(), createUnboundedVarcharType()), "Bob", "Roberts"), 0, "bob.roberts@example.com", - arrayBlockOf(phoneType, rowBlockOf(ImmutableList.of(createUnboundedVarcharType(), createUnboundedVarcharType()), "1234567890", null)) - ) - ) - ) - ); + arrayBlockOf(phoneType, rowBlockOf(ImmutableList.of(createUnboundedVarcharType(), createUnboundedVarcharType()), "1234567890", null)))))); File file = new File(this.getClass().getClassLoader().getResource("addressbook.thrift.lzo").getPath()); FileSplit split = new FileSplit(new Path(file.getAbsolutePath()), 0, file.length(), new String[0]); diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorFactory.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorFactory.java index 88d7410d8c72..b9721f0459f8 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorFactory.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorFactory.java @@ -79,8 +79,7 @@ public Connector create(String connectorId, Map config, Connecto else { binder.bind(new TypeLiteral>>() {}).to(KafkaTableDescriptionSupplier.class).in(Scopes.SINGLETON); } - } - ); + }); Injector injector = app.strictConfig() .doNotInitializeLogging() diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java index bd34eeeaf509..8b2eb7303db5 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java @@ -55,7 +55,7 @@ public class KafkaRecordSet { private static final Logger log = Logger.get(KafkaRecordSet.class); - private static final byte [] EMPTY_BYTE_ARRAY = new byte [0]; + private static final byte[] EMPTY_BYTE_ARRAY = new byte[0]; private final KafkaSplit split; private final KafkaSimpleConsumerManager consumerManager; @@ -139,12 +139,6 @@ public class KafkaRecordCursor this.endTs = endTs; } - @Override - public long getTotalBytes() - { - return totalBytes; - } - @Override public long getCompletedBytes() { diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java index a3f372160af3..1629efcf4f9d 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java @@ -93,7 +93,7 @@ public ConnectorSplitSource getSplits(ConnectorTransactionHandle transaction, Co long startTs = layoutHandle.getOffsetStartTs(); long endTs = layoutHandle.getOffsetEndTs(); - long[] offsets = findAllOffsets(leaderConsumer, kafkaTableHandle.getTopicName(), part.partId(), startTs, endTs); + long[] offsets = findAllOffsets(leaderConsumer, kafkaTableHandle.getTopicName(), part.partId(), startTs, endTs); for (int i = offsets.length - 1; i > 0; i--) { KafkaSplit split = new KafkaSplit( connectorId, diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaConnectorConfig.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaConnectorConfig.java index 06bfbf3d6474..e824778a9bee 100644 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaConnectorConfig.java +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaConnectorConfig.java @@ -34,8 +34,7 @@ public void testDefaults() .setTableDescriptionDir(new File("etc/kafka07/")) .setHideInternalColumns(true) .setFetchSize(10485760) - .setZkEndpoint("") - ); + .setZkEndpoint("")); } @Test diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaDistributed.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaDistributed.java index c6f25cd98f19..c31e40f5d62e 100644 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaDistributed.java +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaDistributed.java @@ -19,6 +19,8 @@ import org.testng.annotations.AfterClass; import org.testng.annotations.Test; +import java.io.IOException; + import static com.facebook.presto.kafka.KafkaQueryRunner.createKafkaQueryRunner; import static com.facebook.presto.kafka.util.EmbeddedKafka.createEmbeddedKafka; @@ -43,6 +45,7 @@ public TestKafkaDistributed(EmbeddedKafka embeddedKafka) @AfterClass(alwaysRun = true) public void destroy() + throws IOException { embeddedKafka.close(); } diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaIntegrationSmokeTest.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaIntegrationSmokeTest.java index 7d73777d22f1..faf02f0bfb80 100644 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaIntegrationSmokeTest.java +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaIntegrationSmokeTest.java @@ -18,6 +18,8 @@ import org.testng.annotations.AfterClass; import org.testng.annotations.Test; +import java.io.IOException; + import static com.facebook.presto.kafka.KafkaQueryRunner.createKafkaQueryRunner; import static com.facebook.presto.kafka.util.EmbeddedKafka.createEmbeddedKafka; import static io.airlift.tpch.TpchTable.ORDERS; @@ -43,6 +45,7 @@ public TestKafkaIntegrationSmokeTest(EmbeddedKafka embeddedKafka) @AfterClass(alwaysRun = true) public void destroy() + throws IOException { embeddedKafka.close(); } diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedKafka.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedKafka.java index 7ba6a35db13e..2ea3a57f3b48 100644 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedKafka.java +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedKafka.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.io.Files; +import com.google.common.io.MoreFiles; import kafka.javaapi.producer.Producer; import kafka.producer.ProducerConfig; import kafka.server.KafkaConfig; @@ -31,7 +32,7 @@ import static com.facebook.presto.kafka.util.TestUtils.findUnusedPort; import static com.facebook.presto.kafka.util.TestUtils.toProperties; -import static io.airlift.testing.FileUtils.deleteRecursively; +import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; import static java.util.Objects.requireNonNull; public class EmbeddedKafka @@ -100,12 +101,13 @@ public void start() @Override public void close() + throws IOException { if (started.get() && !stopped.getAndSet(true)) { kafka.shutdown(); kafka.awaitShutdown(); zookeeper.close(); - deleteRecursively(kafkaDataDir); + MoreFiles.deleteRecursively(kafkaDataDir.toPath(), ALLOW_INSECURE); } } diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedZookeeper.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedZookeeper.java index 5b39deda93e1..d312179421fd 100644 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedZookeeper.java +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedZookeeper.java @@ -14,6 +14,7 @@ package com.facebook.presto.kafka.util; import com.google.common.io.Files; +import com.google.common.io.MoreFiles; import org.apache.zookeeper.server.NIOServerCnxnFactory; import org.apache.zookeeper.server.ServerCnxnFactory; import org.apache.zookeeper.server.ZooKeeperServer; @@ -25,7 +26,7 @@ import java.net.InetSocketAddress; import java.util.concurrent.atomic.AtomicBoolean; -import static io.airlift.testing.FileUtils.deleteRecursively; +import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; public class EmbeddedZookeeper implements Closeable @@ -69,6 +70,7 @@ public void start() @Override public void close() + throws IOException { if (started.get() && !stopped.getAndSet(true)) { cnxnFactory.shutdown(); @@ -83,7 +85,7 @@ public void close() zkServer.shutdown(); } - deleteRecursively(zkDataDir); + MoreFiles.deleteRecursively(zkDataDir.toPath(), ALLOW_INSECURE); } } diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberPartitioner.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberPartitioner.java index 4ab86c73f57c..a1225b358813 100644 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberPartitioner.java +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberPartitioner.java @@ -19,7 +19,8 @@ import static java.lang.Math.toIntExact; -public class NumberPartitioner implements Partitioner +public class NumberPartitioner + implements Partitioner { @Override public int partition(Object key, Seq partitions) diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/TestUtils.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/TestUtils.java index 3c9be963114e..5e194df60de6 100644 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/TestUtils.java +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/TestUtils.java @@ -65,8 +65,7 @@ public static void installKafkaPlugin(EmbeddedKafka embeddedKafka, QueryRunner q "kafka.table-names", Joiner.on(",").join(topicDescriptions.keySet()), "kafka.connect-timeout", "120s", "kafka.default-schema", "default", - "kafka.zk-endpoint", embeddedKafka.getZookeeperConnectString() - ); + "kafka.zk-endpoint", embeddedKafka.getZookeeperConnectString()); queryRunner.createCatalog("kafka", "kafka07", kafkaConfig); } diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftDecoderModule.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftDecoderModule.java index 5135b8175f05..2f919dce9b4b 100644 --- a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftDecoderModule.java +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftDecoderModule.java @@ -20,13 +20,13 @@ import static com.facebook.presto.decoder.DecoderModule.bindFieldDecoder; import static com.facebook.presto.decoder.DecoderModule.bindRowDecoder; -public class ThriftDecoderModule implements Module +public class ThriftDecoderModule + implements Module { - @Override - public void configure(Binder binder) - { - bindRowDecoder(binder, ThriftRowDecoder.class); - - bindFieldDecoder(binder, ThriftFieldDecoder.class); - } + @Override + public void configure(Binder binder) + { + bindRowDecoder(binder, ThriftRowDecoder.class); + bindFieldDecoder(binder, ThriftFieldDecoder.class); + } } diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftFieldDecoder.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftFieldDecoder.java index c62e38b2bb06..c8be21f7c736 100644 --- a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftFieldDecoder.java +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftFieldDecoder.java @@ -20,7 +20,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.StandardTypes; import com.facebook.presto.spi.type.Type; import com.google.common.collect.ImmutableSet; @@ -38,7 +37,7 @@ import static com.facebook.presto.spi.type.BigintType.BIGINT; import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; import static com.facebook.presto.spi.type.Chars.isCharType; -import static com.facebook.presto.spi.type.Chars.trimSpacesAndTruncateToLength; +import static com.facebook.presto.spi.type.Chars.truncateToLengthAndTrimSpaces; import static com.facebook.presto.spi.type.DateType.DATE; import static com.facebook.presto.spi.type.DoubleType.DOUBLE; import static com.facebook.presto.spi.type.IntegerType.INTEGER; @@ -177,7 +176,7 @@ else if (value instanceof Integer) { sliceValue = truncateToLength(sliceValue, type); } if (isCharType(type)) { - sliceValue = trimSpacesAndTruncateToLength(sliceValue, type); + sliceValue = truncateToLengthAndTrimSpaces(sliceValue, type); } return sliceValue; @@ -264,7 +263,7 @@ private static Block serializeMap(Type type, BlockBuilder builder, Object object currentBuilder = builder.beginBlockEntry(); } else { - currentBuilder = new InterleavedBlockBuilder(typeParameters, new BlockBuilderStatus(), map.size()); + currentBuilder = type.createBlockBuilder(new BlockBuilderStatus(), map.size()); } for (Map.Entry entry : map.entrySet()) { @@ -299,7 +298,7 @@ private static Block serializeStruct(Type type, BlockBuilder builder, Object obj currentBuilder = builder.beginBlockEntry(); } else { - currentBuilder = new InterleavedBlockBuilder(typeParameters, new BlockBuilderStatus(), typeParameters.size()); + currentBuilder = type.createBlockBuilder(new BlockBuilderStatus(), typeParameters.size()); } for (int i = 0; i < typeParameters.size(); i++) { diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftGenericRow.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftGenericRow.java index d4061030e108..53e25894583a 100644 --- a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftGenericRow.java +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftGenericRow.java @@ -37,7 +37,8 @@ import java.util.Map; import java.util.Set; -public class ThriftGenericRow implements TBase +public class ThriftGenericRow + implements TBase { private static final Logger log = Logger.get(ThriftGenericRow.class); private final Map values = new HashMap<>(); @@ -45,14 +46,17 @@ public class ThriftGenericRow implements TBase values) { this.values.putAll(values); } - public class Fields implements TFieldIdEnum + public class Fields + implements TFieldIdEnum { private final short thriftId; private final String fieldName; @@ -74,7 +78,8 @@ public String getFieldName() } } - public void read(TProtocol iprot) throws TException + public void read(TProtocol iprot) + throws TException { TTransport trans = iprot.getTransport(); buf = trans.getBuffer(); @@ -83,12 +88,14 @@ public void read(TProtocol iprot) throws TException len = trans.getBufferPosition() - off; } - public void parse() throws TException + public void parse() + throws TException { parse(null); } - public void parse(short[] thriftIds) throws TException + public void parse(short[] thriftIds) + throws TException { Set idSet = thriftIds == null ? null : new HashSet(Arrays.asList(ArrayUtils.toObject(thriftIds))); TMemoryInputTransport trans = new TMemoryInputTransport(buf, off, len); @@ -111,7 +118,8 @@ public void parse(short[] thriftIds) throws TException iprot.readStructEnd(); } - private Object readElem(TProtocol iprot, byte type) throws TException + private Object readElem(TProtocol iprot, byte type) + throws TException { switch (type) { case TType.BOOL: @@ -143,7 +151,8 @@ private Object readElem(TProtocol iprot, byte type) throws TException } } - private Object readStruct(TProtocol iprot) throws TException + private Object readStruct(TProtocol iprot) + throws TException { ThriftGenericRow elem = new ThriftGenericRow(); elem.read(iprot); @@ -151,33 +160,36 @@ private Object readStruct(TProtocol iprot) throws TException return elem; } - private Object readList(TProtocol iprot) throws TException + private Object readList(TProtocol iprot) + throws TException { TList ilist = iprot.readListBegin(); List listValue = new ArrayList<>(); - for (int i = 0; i < ilist.size; ++i) { + for (int i = 0; i < ilist.size; i++) { listValue.add(readElem(iprot, ilist.elemType)); } iprot.readListEnd(); return listValue; } - private Object readSet(TProtocol iprot) throws TException + private Object readSet(TProtocol iprot) + throws TException { TSet iset = iprot.readSetBegin(); List setValue = new ArrayList<>(); - for (int i = 0; i < iset.size; ++i) { + for (int i = 0; i < iset.size; i++) { setValue.add(readElem(iprot, iset.elemType)); } iprot.readSetEnd(); return setValue; } - private Object readMap(TProtocol iprot) throws TException + private Object readMap(TProtocol iprot) + throws TException { TMap imap = iprot.readMapBegin(); Map mapValue = new HashMap<>(); - for (int i = 0; i < imap.size; ++i) { + for (int i = 0; i < imap.size; i++) { mapValue.put(readElem(iprot, imap.keyType), readElem(iprot, imap.valueType)); } iprot.readMapEnd(); @@ -194,7 +206,9 @@ public ThriftGenericRow deepCopy() return new ThriftGenericRow(values); } - public void clear() {} + public void clear() + { + } public Fields fieldForId(int fieldId) { @@ -216,7 +230,8 @@ public void setFieldValue(Fields field, Object value) values.put(field.getThriftFieldId(), value); } - public void write(TProtocol oprot) throws TException + public void write(TProtocol oprot) + throws TException { throw new UnsupportedOperationException("ThriftGenericRow.write is not supported."); } diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java index f6e13956b4dc..4a9376fa7290 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java @@ -34,83 +34,83 @@ */ public class QueryCompletedEventScriber { - private static final String DASH = "-"; - private static final Logger log = Logger.get(QueryCompletedEventScriber.class); + private static final String DASH = "-"; + private static final Logger log = Logger.get(QueryCompletedEventScriber.class); - private TwitterScriber scriber = new TwitterScriber("presto_query_completion"); + private TwitterScriber scriber = new TwitterScriber("presto_query_completion"); - public void handle(QueryCompletedEvent event) - { - try { - scriber.scribe(toThriftQueryCompletionEvent(event)); + public void handle(QueryCompletedEvent event) + { + try { + scriber.scribe(toThriftQueryCompletionEvent(event)); + } + catch (TException e) { + log.warn(e, + String.format("Could not serialize thrift object of Query(id=%s, user=%s, env=%s, schema=%s.%s)", + event.getMetadata().getQueryId(), + event.getContext().getUser(), + event.getContext().getEnvironment(), + event.getContext().getCatalog().orElse(DASH), + event.getContext().getSchema().orElse(DASH))); + } } - catch (TException e) { - log.warn(e, - String.format("Could not serialize thrift object of Query(id=%s, user=%s, env=%s, schema=%s.%s)", - event.getMetadata().getQueryId(), - event.getContext().getUser(), - event.getContext().getEnvironment(), - event.getContext().getCatalog().orElse(DASH), - event.getContext().getSchema().orElse(DASH))); - } - } - private static QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletedEvent event) - { - QueryMetadata eventMetadata = event.getMetadata(); - QueryContext eventContext = event.getContext(); - QueryStatistics eventStat = event.getStatistics(); + private static QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletedEvent event) + { + QueryMetadata eventMetadata = event.getMetadata(); + QueryContext eventContext = event.getContext(); + QueryStatistics eventStat = event.getStatistics(); - QueryCompletionEvent thriftEvent = - new com.twitter.presto.thriftjava.QueryCompletionEvent(); + QueryCompletionEvent thriftEvent = + new com.twitter.presto.thriftjava.QueryCompletionEvent(); - thriftEvent.query_id = eventMetadata.getQueryId(); - thriftEvent.transaction_id = eventMetadata.getTransactionId().orElse(DASH); - thriftEvent.user = eventContext.getUser(); - thriftEvent.principal = eventContext.getPrincipal().orElse(DASH); - thriftEvent.source = eventContext.getSource().orElse(DASH); - thriftEvent.server_version = eventContext.getServerVersion(); - thriftEvent.environment = eventContext.getEnvironment(); - thriftEvent.catalog = eventContext.getCatalog().orElse(DASH); - thriftEvent.schema = eventContext.getSchema().orElse(DASH); - Map> queriedColumnsByTable = new HashMap>(); - event.getIoMetadata().getInputs().forEach(input -> queriedColumnsByTable.put(String.format("%s.%s", input.getSchema(), input.getTable()), input.getColumns())); - thriftEvent.queried_columns_by_table = queriedColumnsByTable; - thriftEvent.remote_client_address = eventContext.getRemoteClientAddress().orElse(DASH); - thriftEvent.user_agent = eventContext.getUserAgent().orElse(DASH); - thriftEvent.query_state = QueryState.valueOf(eventMetadata.getQueryState()); - thriftEvent.uri = eventMetadata.getUri().toString(); - thriftEvent.query = eventMetadata.getQuery(); - thriftEvent.create_time_ms = event.getCreateTime().toEpochMilli(); - thriftEvent.execution_start_time_ms = event.getExecutionStartTime().toEpochMilli(); - thriftEvent.end_time_ms = event.getEndTime().toEpochMilli(); - thriftEvent.queued_time_ms = eventStat.getQueuedTime().toMillis(); - thriftEvent.query_wall_time_ms = eventStat.getWallTime().toMillis(); - thriftEvent.cumulative_memory_bytesecond = eventStat.getCumulativeMemory(); - thriftEvent.peak_memory_bytes = eventStat.getPeakMemoryBytes(); - thriftEvent.cpu_time_ms = eventStat.getCpuTime().toMillis(); - if (eventStat.getAnalysisTime().isPresent()) { - thriftEvent.analysis_time_ms = eventStat.getAnalysisTime().get().toMillis(); - } - if (eventStat.getDistributedPlanningTime().isPresent()) { - thriftEvent.distributed_planning_time_ms = eventStat.getDistributedPlanningTime().get().toMillis(); - } - thriftEvent.total_bytes = eventStat.getTotalBytes(); - thriftEvent.query_stages = QueryStatsHelper.getQueryStages(eventMetadata); - thriftEvent.operator_summaries = QueryStatsHelper.getOperatorSummaries(eventStat); - thriftEvent.total_rows = eventStat.getTotalRows(); - thriftEvent.splits = eventStat.getCompletedSplits(); - if (event.getFailureInfo().isPresent()) { - QueryFailureInfo eventFailureInfo = event.getFailureInfo().get(); - thriftEvent.error_code_id = eventFailureInfo.getErrorCode().getCode(); - thriftEvent.error_code_name = eventFailureInfo.getErrorCode().getName(); - thriftEvent.failure_type = eventFailureInfo.getFailureType().orElse(DASH); - thriftEvent.failure_message = eventFailureInfo.getFailureMessage().orElse(DASH); - thriftEvent.failure_task = eventFailureInfo.getFailureTask().orElse(DASH); - thriftEvent.failure_host = eventFailureInfo.getFailureHost().orElse(DASH); - thriftEvent.failures_json = eventFailureInfo.getFailuresJson(); - } + thriftEvent.query_id = eventMetadata.getQueryId(); + thriftEvent.transaction_id = eventMetadata.getTransactionId().orElse(DASH); + thriftEvent.user = eventContext.getUser(); + thriftEvent.principal = eventContext.getPrincipal().orElse(DASH); + thriftEvent.source = eventContext.getSource().orElse(DASH); + thriftEvent.server_version = eventContext.getServerVersion(); + thriftEvent.environment = eventContext.getEnvironment(); + thriftEvent.catalog = eventContext.getCatalog().orElse(DASH); + thriftEvent.schema = eventContext.getSchema().orElse(DASH); + Map> queriedColumnsByTable = new HashMap>(); + event.getIoMetadata().getInputs().forEach(input -> queriedColumnsByTable.put(String.format("%s.%s", input.getSchema(), input.getTable()), input.getColumns())); + thriftEvent.queried_columns_by_table = queriedColumnsByTable; + thriftEvent.remote_client_address = eventContext.getRemoteClientAddress().orElse(DASH); + thriftEvent.user_agent = eventContext.getUserAgent().orElse(DASH); + thriftEvent.query_state = QueryState.valueOf(eventMetadata.getQueryState()); + thriftEvent.uri = eventMetadata.getUri().toString(); + thriftEvent.query = eventMetadata.getQuery(); + thriftEvent.create_time_ms = event.getCreateTime().toEpochMilli(); + thriftEvent.execution_start_time_ms = event.getExecutionStartTime().toEpochMilli(); + thriftEvent.end_time_ms = event.getEndTime().toEpochMilli(); + thriftEvent.queued_time_ms = eventStat.getQueuedTime().toMillis(); + thriftEvent.query_wall_time_ms = eventStat.getWallTime().toMillis(); + thriftEvent.cumulative_memory_bytesecond = eventStat.getCumulativeMemory(); + thriftEvent.peak_memory_bytes = eventStat.getPeakMemoryBytes(); + thriftEvent.cpu_time_ms = eventStat.getCpuTime().toMillis(); + if (eventStat.getAnalysisTime().isPresent()) { + thriftEvent.analysis_time_ms = eventStat.getAnalysisTime().get().toMillis(); + } + if (eventStat.getDistributedPlanningTime().isPresent()) { + thriftEvent.distributed_planning_time_ms = eventStat.getDistributedPlanningTime().get().toMillis(); + } + thriftEvent.total_bytes = eventStat.getTotalBytes(); + thriftEvent.query_stages = QueryStatsHelper.getQueryStages(eventMetadata); + thriftEvent.operator_summaries = QueryStatsHelper.getOperatorSummaries(eventStat); + thriftEvent.total_rows = eventStat.getTotalRows(); + thriftEvent.splits = eventStat.getCompletedSplits(); + if (event.getFailureInfo().isPresent()) { + QueryFailureInfo eventFailureInfo = event.getFailureInfo().get(); + thriftEvent.error_code_id = eventFailureInfo.getErrorCode().getCode(); + thriftEvent.error_code_name = eventFailureInfo.getErrorCode().getName(); + thriftEvent.failure_type = eventFailureInfo.getFailureType().orElse(DASH); + thriftEvent.failure_message = eventFailureInfo.getFailureMessage().orElse(DASH); + thriftEvent.failure_task = eventFailureInfo.getFailureTask().orElse(DASH); + thriftEvent.failure_host = eventFailureInfo.getFailureHost().orElse(DASH); + thriftEvent.failures_json = eventFailureInfo.getFailuresJson(); + } - return thriftEvent; - } + return thriftEvent; + } } diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java index 0e9431a0e6c2..62cb231075b0 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java @@ -39,181 +39,181 @@ public class QueryStatsHelper { - private static final Logger log = Logger.get(QueryStatsHelper.class); + private static final Logger log = Logger.get(QueryStatsHelper.class); - private QueryStatsHelper() - { - throw new AssertionError(); - } - - private static long getBytesOrNegativeOne(String strVal) - { - try { - return DataSize.valueOf(strVal).toBytes(); - } - catch (IllegalArgumentException e) { - log.warn(e, - String.format("Failed to parse io.airlift.units.DataSize '%s', returning -1", strVal)); - return -1; + private QueryStatsHelper() + { + throw new AssertionError(); } - } - private static long getMillisOrNegativeOne(String strVal) - { - try { - return Duration.valueOf(strVal).toMillis(); - } - catch (IllegalArgumentException e) { - log.warn(e, - String.format("Failed to parse io.airlift.units.Duration '%s', returning -1", strVal)); - return -1; - } - } - - private static QueryStageInfo getQueryStageInfo(int stageId, JsonObject stage) - { - QueryStageInfo stageInfo = new QueryStageInfo(); - - stageInfo.stage_id = stageId; - try { - JsonObject stageStats = stage.getJsonObject("stageStats"); - stageInfo.raw_input_data_size_bytes = getBytesOrNegativeOne(stageStats.getString("rawInputDataSize")); - stageInfo.output_data_size_bytes = getBytesOrNegativeOne(stageStats.getString("outputDataSize")); - stageInfo.completed_tasks = stageStats.getInt("completedTasks"); - stageInfo.completed_drivers = stageStats.getInt("completedDrivers"); - stageInfo.cumulative_memory = stageStats.getJsonNumber("cumulativeMemory").doubleValue(); - stageInfo.peak_memory_reservation_bytes = getBytesOrNegativeOne(stageStats.getString("peakMemoryReservation")); - stageInfo.total_scheduled_time_millis = getMillisOrNegativeOne(stageStats.getString("totalScheduledTime")); - stageInfo.total_cpu_time_millis = getMillisOrNegativeOne(stageStats.getString("totalCpuTime")); - stageInfo.total_user_time_millis = getMillisOrNegativeOne(stageStats.getString("totalUserTime")); - stageInfo.total_blocked_time_millis = getMillisOrNegativeOne(stageStats.getString("totalBlockedTime")); + private static long getBytesOrNegativeOne(String strVal) + { + try { + return DataSize.valueOf(strVal).toBytes(); + } + catch (IllegalArgumentException e) { + log.warn(e, + String.format("Failed to parse io.airlift.units.DataSize '%s', returning -1", strVal)); + return -1; + } } - catch (Exception e) { - log.error(e, String.format("Error retrieving stage stats for stage %d", stageId)); - return null; + + private static long getMillisOrNegativeOne(String strVal) + { + try { + return Duration.valueOf(strVal).toMillis(); + } + catch (IllegalArgumentException e) { + log.warn(e, + String.format("Failed to parse io.airlift.units.Duration '%s', returning -1", strVal)); + return -1; + } } - return stageInfo; - } + private static QueryStageInfo getQueryStageInfo(int stageId, JsonObject stage) + { + QueryStageInfo stageInfo = new QueryStageInfo(); + + stageInfo.stage_id = stageId; + try { + JsonObject stageStats = stage.getJsonObject("stageStats"); + stageInfo.raw_input_data_size_bytes = getBytesOrNegativeOne(stageStats.getString("rawInputDataSize")); + stageInfo.output_data_size_bytes = getBytesOrNegativeOne(stageStats.getString("outputDataSize")); + stageInfo.completed_tasks = stageStats.getInt("completedTasks"); + stageInfo.completed_drivers = stageStats.getInt("completedDrivers"); + stageInfo.cumulative_memory = stageStats.getJsonNumber("cumulativeMemory").doubleValue(); + stageInfo.peak_memory_reservation_bytes = getBytesOrNegativeOne(stageStats.getString("peakMemoryReservation")); + stageInfo.total_scheduled_time_millis = getMillisOrNegativeOne(stageStats.getString("totalScheduledTime")); + stageInfo.total_cpu_time_millis = getMillisOrNegativeOne(stageStats.getString("totalCpuTime")); + stageInfo.total_user_time_millis = getMillisOrNegativeOne(stageStats.getString("totalUserTime")); + stageInfo.total_blocked_time_millis = getMillisOrNegativeOne(stageStats.getString("totalBlockedTime")); + } + catch (Exception e) { + log.error(e, String.format("Error retrieving stage stats for stage %d", stageId)); + return null; + } - private static OperatorStats getOperatorStat(String operatorSummaryStr) - { - try { - JsonReader jsonReader = Json.createReader(new StringReader(operatorSummaryStr)); - return getOperatorStat(jsonReader.readObject()); - } - catch (Exception e) { - log.error(e, String.format("Error retrieving operator stats from string:\n%s\n", operatorSummaryStr)); + return stageInfo; } - return null; - } - - private static OperatorStats getOperatorStat(JsonObject obj) - { - OperatorStats operatorStats = new OperatorStats(); - - try { - operatorStats.pipeline_id = obj.getInt("pipelineId"); - operatorStats.operator_id = obj.getInt("operatorId"); - operatorStats.plan_node_id = obj.getString("planNodeId"); - operatorStats.operator_type = obj.getString("operatorType"); - operatorStats.total_drivers = obj.getJsonNumber("totalDrivers").longValue(); - operatorStats.add_input_calls = obj.getJsonNumber("addInputCalls").longValue(); - operatorStats.add_input_wall_millis = getMillisOrNegativeOne(obj.getString("addInputWall")); - operatorStats.add_input_cpu_millis = getMillisOrNegativeOne(obj.getString("addInputCpu")); - operatorStats.add_input_user_millis = getMillisOrNegativeOne(obj.getString("addInputUser")); - operatorStats.input_data_size_bytes = getBytesOrNegativeOne(obj.getString("inputDataSize")); - operatorStats.input_positions = obj.getJsonNumber("inputPositions").longValue(); - operatorStats.sum_squared_input_positions = obj.getJsonNumber("sumSquaredInputPositions").doubleValue(); - operatorStats.get_output_calls = obj.getJsonNumber("getOutputCalls").longValue(); - operatorStats.get_output_wall_millis = getMillisOrNegativeOne(obj.getString("getOutputWall")); - operatorStats.get_output_cpu_millis = getMillisOrNegativeOne(obj.getString("getOutputCpu")); - operatorStats.get_output_user_millis = getMillisOrNegativeOne(obj.getString("getOutputUser")); - operatorStats.output_data_size_bytes = getBytesOrNegativeOne(obj.getString("outputDataSize")); - operatorStats.output_positions = obj.getJsonNumber("outputPositions").longValue(); - operatorStats.blocked_wall_millis = getMillisOrNegativeOne(obj.getString("blockedWall")); - operatorStats.finish_calls = obj.getJsonNumber("finishCalls").longValue(); - operatorStats.finish_wall_millis = getMillisOrNegativeOne(obj.getString("finishWall")); - operatorStats.finish_cpu_millis = getMillisOrNegativeOne(obj.getString("finishCpu")); - operatorStats.finish_user_millis = getMillisOrNegativeOne(obj.getString("finishUser")); - operatorStats.memory_reservation_bytes = getBytesOrNegativeOne(obj.getString("memoryReservation")); - operatorStats.system_memory_reservation_bytes = getBytesOrNegativeOne(obj.getString("systemMemoryReservation")); - } - catch (Exception e) { - log.error(e, String.format("Error retrieving operator stats from JsonObject:\n%s\n", obj.toString())); - return null; + private static OperatorStats getOperatorStat(String operatorSummaryStr) + { + try { + JsonReader jsonReader = Json.createReader(new StringReader(operatorSummaryStr)); + return getOperatorStat(jsonReader.readObject()); + } + catch (Exception e) { + log.error(e, String.format("Error retrieving operator stats from string:\n%s\n", operatorSummaryStr)); + } + + return null; } - return operatorStats; - } + private static OperatorStats getOperatorStat(JsonObject obj) + { + OperatorStats operatorStats = new OperatorStats(); + + try { + operatorStats.pipeline_id = obj.getInt("pipelineId"); + operatorStats.operator_id = obj.getInt("operatorId"); + operatorStats.plan_node_id = obj.getString("planNodeId"); + operatorStats.operator_type = obj.getString("operatorType"); + operatorStats.total_drivers = obj.getJsonNumber("totalDrivers").longValue(); + operatorStats.add_input_calls = obj.getJsonNumber("addInputCalls").longValue(); + operatorStats.add_input_wall_millis = getMillisOrNegativeOne(obj.getString("addInputWall")); + operatorStats.add_input_cpu_millis = getMillisOrNegativeOne(obj.getString("addInputCpu")); + operatorStats.add_input_user_millis = getMillisOrNegativeOne(obj.getString("addInputUser")); + operatorStats.input_data_size_bytes = getBytesOrNegativeOne(obj.getString("inputDataSize")); + operatorStats.input_positions = obj.getJsonNumber("inputPositions").longValue(); + operatorStats.sum_squared_input_positions = obj.getJsonNumber("sumSquaredInputPositions").doubleValue(); + operatorStats.get_output_calls = obj.getJsonNumber("getOutputCalls").longValue(); + operatorStats.get_output_wall_millis = getMillisOrNegativeOne(obj.getString("getOutputWall")); + operatorStats.get_output_cpu_millis = getMillisOrNegativeOne(obj.getString("getOutputCpu")); + operatorStats.get_output_user_millis = getMillisOrNegativeOne(obj.getString("getOutputUser")); + operatorStats.output_data_size_bytes = getBytesOrNegativeOne(obj.getString("outputDataSize")); + operatorStats.output_positions = obj.getJsonNumber("outputPositions").longValue(); + operatorStats.blocked_wall_millis = getMillisOrNegativeOne(obj.getString("blockedWall")); + operatorStats.finish_calls = obj.getJsonNumber("finishCalls").longValue(); + operatorStats.finish_wall_millis = getMillisOrNegativeOne(obj.getString("finishWall")); + operatorStats.finish_cpu_millis = getMillisOrNegativeOne(obj.getString("finishCpu")); + operatorStats.finish_user_millis = getMillisOrNegativeOne(obj.getString("finishUser")); + operatorStats.memory_reservation_bytes = getBytesOrNegativeOne(obj.getString("memoryReservation")); + operatorStats.system_memory_reservation_bytes = getBytesOrNegativeOne(obj.getString("systemMemoryReservation")); + } + catch (Exception e) { + log.error(e, String.format("Error retrieving operator stats from JsonObject:\n%s\n", obj.toString())); + return null; + } - public static Map getQueryStages(QueryMetadata eventMetadata) - { - if (!eventMetadata.getPayload().isPresent()) { - return null; + return operatorStats; } - String payload = eventMetadata.getPayload().get(); - Queue stageJsonObjs = new LinkedList(); - try { - JsonReader jsonReader = Json.createReader(new StringReader(payload)); - stageJsonObjs.add(jsonReader.readObject()); - } - catch (Exception e) { - log.error(e, - String.format("getQueryStages - Unable to extract JsonObject out of following blob:\n%s\n", payload)); - return null; - } + public static Map getQueryStages(QueryMetadata eventMetadata) + { + if (!eventMetadata.getPayload().isPresent()) { + return null; + } - Map stages = new HashMap(); - while (!stageJsonObjs.isEmpty()) { - JsonObject cur = stageJsonObjs.poll(); - String stageIdStr = "Unknown"; - try { - stageIdStr = cur.getString("stageId"); - int stageId = Integer.parseInt(stageIdStr.split("\\.")[1]); - QueryStageInfo curStage = getQueryStageInfo(stageId, cur); - if (curStage != null) { - stages.put(stageId, getQueryStageInfo(stageId, cur)); - } - } - catch (Exception e) { - log.error(e, - String.format("Failed to parse QueryStageInfo from JsonObject:\n%s\n", cur.toString())); - return null; - } - - try { - cur.getJsonArray("subStages") - .stream() - .filter(val -> val.getValueType() == ValueType.OBJECT) - .forEach(val -> stageJsonObjs.add((JsonObject) val)); - } - catch (Exception e) { - log.error(e, - String.format("Failed to get subStages for stage %s, treating as no subStages", stageIdStr)); - } - } + String payload = eventMetadata.getPayload().get(); + Queue stageJsonObjs = new LinkedList(); + try { + JsonReader jsonReader = Json.createReader(new StringReader(payload)); + stageJsonObjs.add(jsonReader.readObject()); + } + catch (Exception e) { + log.error(e, + String.format("getQueryStages - Unable to extract JsonObject out of following blob:\n%s\n", payload)); + return null; + } - return stages; - } - - public static List getOperatorSummaries(QueryStatistics eventStat) - { - try { - return eventStat.getOperatorSummaries() - .stream() - .filter(val -> val != null && !val.isEmpty()) - .map(QueryStatsHelper::getOperatorStat) - .filter(Objects::nonNull) - .collect(Collectors.toList()); - } - catch (Exception e) { - log.error(e, - String.format("Error converting List to List:\n%s\n", eventStat.getOperatorSummaries().toString())); + Map stages = new HashMap(); + while (!stageJsonObjs.isEmpty()) { + JsonObject cur = stageJsonObjs.poll(); + String stageIdStr = "Unknown"; + try { + stageIdStr = cur.getString("stageId"); + int stageId = Integer.parseInt(stageIdStr.split("\\.")[1]); + QueryStageInfo curStage = getQueryStageInfo(stageId, cur); + if (curStage != null) { + stages.put(stageId, getQueryStageInfo(stageId, cur)); + } + } + catch (Exception e) { + log.error(e, + String.format("Failed to parse QueryStageInfo from JsonObject:\n%s\n", cur.toString())); + return null; + } + + try { + cur.getJsonArray("subStages") + .stream() + .filter(val -> val.getValueType() == ValueType.OBJECT) + .forEach(val -> stageJsonObjs.add((JsonObject) val)); + } + catch (Exception e) { + log.error(e, + String.format("Failed to get subStages for stage %s, treating as no subStages", stageIdStr)); + } + } + + return stages; } - return null; - } + public static List getOperatorSummaries(QueryStatistics eventStat) + { + try { + return eventStat.getOperatorSummaries() + .stream() + .filter(val -> val != null && !val.isEmpty()) + .map(QueryStatsHelper::getOperatorStat) + .filter(Objects::nonNull) + .collect(Collectors.toList()); + } + catch (Exception e) { + log.error(e, + String.format("Error converting List to List:\n%s\n", eventStat.getOperatorSummaries().toString())); + } + + return null; + } } diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java index 0c622b0ba34c..ef9e0ed2b36c 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java @@ -20,24 +20,25 @@ import io.airlift.log.Logger; -public class TwitterEventListener implements EventListener +public class TwitterEventListener + implements EventListener { - private static final Logger log = Logger.get(TwitterEventListener.class); - private final QueryCompletedEventScriber scriber = new QueryCompletedEventScriber(); + private static final Logger log = Logger.get(TwitterEventListener.class); + private final QueryCompletedEventScriber scriber = new QueryCompletedEventScriber(); - @Override - public void queryCreated(QueryCreatedEvent queryCreatedEvent) - { - } + @Override + public void queryCreated(QueryCreatedEvent queryCreatedEvent) + { + } - @Override - public void queryCompleted(QueryCompletedEvent queryCompletedEvent) - { - scriber.handle(queryCompletedEvent); - } + @Override + public void queryCompleted(QueryCompletedEvent queryCompletedEvent) + { + scriber.handle(queryCompletedEvent); + } - @Override - public void splitCompleted(SplitCompletedEvent splitCompletedEvent) - { - } + @Override + public void splitCompleted(SplitCompletedEvent splitCompletedEvent) + { + } } diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerFactory.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerFactory.java index 6fedc2d780f1..a22762c9ec32 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerFactory.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerFactory.java @@ -18,17 +18,18 @@ import java.util.Map; -public class TwitterEventListenerFactory implements EventListenerFactory +public class TwitterEventListenerFactory + implements EventListenerFactory { - @Override - public String getName() - { - return "twitter-event-listener"; - } + @Override + public String getName() + { + return "twitter-event-listener"; + } - @Override - public EventListener create(Map config) - { - return new TwitterEventListener(); - } + @Override + public EventListener create(Map config) + { + return new TwitterEventListener(); + } } diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerPlugin.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerPlugin.java index 557256058490..5caf7b8724a9 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerPlugin.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerPlugin.java @@ -18,11 +18,12 @@ import com.google.common.collect.ImmutableList; -public class TwitterEventListenerPlugin implements Plugin +public class TwitterEventListenerPlugin + implements Plugin { - @Override - public Iterable getEventListenerFactories() - { - return ImmutableList.of(new TwitterEventListenerFactory()); - } + @Override + public Iterable getEventListenerFactories() + { + return ImmutableList.of(new TwitterEventListenerFactory()); + } } diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java index 9e8524602255..8acc7ca0ad9e 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java @@ -27,52 +27,54 @@ public class TwitterScriber { - private static final String DASH = "-"; - private static final int MAX_QUEUE_SIZE = 1000; + private static final String DASH = "-"; + private static final int MAX_QUEUE_SIZE = 1000; - private QueueingHandler queueingHandler; + private QueueingHandler queueingHandler; - // TSerializer is not thread safe - private final ThreadLocal serializer = new ThreadLocal() - { - @Override protected TSerializer initialValue() + // TSerializer is not thread safe + private final ThreadLocal serializer = new ThreadLocal() { + @Override + protected TSerializer initialValue() + { + return new TSerializer(); + } + }; + + public TwitterScriber(String scribeCategory) { - return new TSerializer(); + ScribeHandler scribeHandler = new ScribeHandler( + ScribeHandler.DefaultHostname(), + ScribeHandler.DefaultPort(), + scribeCategory, + ScribeHandler.DefaultBufferTime(), + ScribeHandler.DefaultConnectBackoff(), + ScribeHandler.DefaultMaxMessagesPerTransaction(), + ScribeHandler.DefaultMaxMessagesToBuffer(), + BareFormatter$.MODULE$, + scala.Option.apply((Level) null)); + queueingHandler = new QueueingHandler(scribeHandler, MAX_QUEUE_SIZE); } - }; - - public TwitterScriber(String scribeCategory) - { - ScribeHandler scribeHandler = new ScribeHandler( - ScribeHandler.DefaultHostname(), - ScribeHandler.DefaultPort(), - scribeCategory, - ScribeHandler.DefaultBufferTime(), - ScribeHandler.DefaultConnectBackoff(), - ScribeHandler.DefaultMaxMessagesPerTransaction(), - ScribeHandler.DefaultMaxMessagesToBuffer(), - BareFormatter$.MODULE$, - scala.Option.apply((Level) null)); - queueingHandler = new QueueingHandler(scribeHandler, MAX_QUEUE_SIZE); - } - public void scribe(TBase thriftMessage) throws TException - { - scribe(serializeThriftToString(thriftMessage)); - } + public void scribe(TBase thriftMessage) + throws TException + { + scribe(serializeThriftToString(thriftMessage)); + } - /** - * Serialize a thrift object to bytes, compress, then encode as a base64 string. - * Throws TException - */ - private String serializeThriftToString(TBase thriftMessage) throws TException - { - return Base64.getEncoder().encodeToString(serializer.get().serialize(thriftMessage)); - } + /** + * Serialize a thrift object to bytes, compress, then encode as a base64 string. + * Throws TException + */ + private String serializeThriftToString(TBase thriftMessage) + throws TException + { + return Base64.getEncoder().encodeToString(serializer.get().serialize(thriftMessage)); + } - private void scribe(String message) - { - LogRecord logRecord = new LogRecord(Level.ALL, message); - queueingHandler.publish(logRecord); - } + private void scribe(String message) + { + LogRecord logRecord = new LogRecord(Level.ALL, message); + queueingHandler.publish(logRecord); + } } From c92d55d5a9cf43f75e1d206a018d72c126f148de Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 18 Oct 2017 22:05:56 -0700 Subject: [PATCH 231/331] downgrade curator-test version --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index d806b791f6ed..ef91ee22d39a 100644 --- a/pom.xml +++ b/pom.xml @@ -891,7 +891,7 @@ org.apache.curator curator-test - 4.0.0 + 2.12.0 org.apache.zookeeper From e86c049f45c27d795fe02a81568278d00fb4aa93 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 18 Oct 2017 23:43:00 -0700 Subject: [PATCH 232/331] Fix tests for BackgroundHiveSplitLoader --- .../com/facebook/presto/hive/BackgroundHiveSplitLoader.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) 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 abeb06c3f851..6b6d9e8842a2 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 @@ -641,6 +641,9 @@ protected HiveSplit computeNext() } blockLocationIterator.next(); chunkOffset -= blockLocation.getLength(); + if (chunkOffset == 0) { + break; + } if (blockLocationIterator.hasNext()) { blockLocation = blockLocationIterator.peek(); } @@ -744,7 +747,7 @@ private static Optional getPathDomain(TupleDomain effe private static boolean pathMatchesPredicate(Optional pathDomain, String path) { if (isLzopIndexFile(new Path(path))) { - return true; + return false; } if (!pathDomain.isPresent()) { From ba6c399551ce95de66bc2350559a086c1923a0af Mon Sep 17 00:00:00 2001 From: Nezih Yigitbasi Date: Thu, 19 Oct 2017 11:01:16 -0700 Subject: [PATCH 233/331] [maven-release-plugin] prepare release 0.187 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- 52 files changed, 53 insertions(+), 53 deletions(-) diff --git a/pom.xml b/pom.xml index 9603639b6e93..d4f8f78831ff 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/facebook/presto.git https://github.com/facebook/presto - HEAD + 0.187 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index f9dc5ce7610d..2f2c0ad5b65e 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 04ee352382d6..0895ff78e0d3 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index a0926fddf9fb..c3736ed0b498 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 79b927d962ce..fa135d9905e0 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 5f136a66caef..57dc61b6d1bc 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 98082ea2fbc2..f0cbb1a31e5b 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.187-SNAPSHOT + 0.187 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index b41b8d198812..68825acac829 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 5a759555771b..32405b841d48 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index ba898fa2e431..1f75449ea02f 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index f045cf63758e..859a397becee 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index b2a5e14444a1..8bc4461bc7a6 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 39986aa1ff17..d3e9debbba56 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 0664034b47c3..77d75a0b5084 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index eba3f89e4d19..ec626fc9bce1 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-example-http diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index 79ef3268e257..314dc85685f9 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 085ada331dfb..e9d6501964c6 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 5313a590e7a6..cbdb01d63c74 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 06d0af27ff7e..8b7c806532b1 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 9ebea00ae390..0292d930601f 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 5a3eda138989..434a0edd4464 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-kafka diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 620179fad384..095d94b5526d 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index c6684c59613c..986548d69134 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index e03a307c86da..0ba9bd640b3c 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.187-SNAPSHOT + 0.187 presto-matching diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 86cc735dce10..717a7a0d05ce 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 4097bfbcb88b..7af1cedb6c50 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 67b847e9ee26..7e3ea5b3f067 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 2cdee770b231..5f0cb15ce6fe 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index ceb846e75c51..ea3d33b6b079 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index e2e0dd0b1184..98afecfeec0b 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 310370f3c646..e57faadc4b66 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index f6a6b122518d..0ac4cb24b971 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 0a49ca7c0a38..af1ade90e4d6 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.187-SNAPSHOT + 0.187 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index d8d9ec75bc4d..989c2718b1c3 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 6731300c6ece..d93e70d6a898 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index daac1e6f01f0..ec7aadd26874 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 6ff52e6e514f..270aeeb6539a 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index 096ec023ffcc..2cabd896e1e1 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index aea3a32375f4..698342827ed5 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index c2a1236de08b..4c02c87be26f 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 5c71c5cbe773..25b961d3ef75 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index f653402dcd5d..82e989a02963 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 23aa9c7e273d..08534be2d256 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.187-SNAPSHOT + 0.187 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 9fe2cdaf5475..a66309b2967c 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 1b253b9dafe9..70c53762ee99 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 81a9262f7cd6..486c1a241cd5 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.187-SNAPSHOT + 0.187 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index 4be473811291..971831f9f68d 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 7a6ef134a74b..7809b2245aaa 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index 3e3dcfbac2f4..cd6283e4e9b4 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index a66da52a4a12..2aeda55c8bdd 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 41023f10eddb..57b8d1c2edca 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-tpch diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 7b4cd9fcd4f6..6f35edee3e74 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-SNAPSHOT + 0.187 presto-verifier From 076fa7f6d11dc8717db0ae0f415b845b6b231bb4 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 30 Aug 2017 03:09:12 -0700 Subject: [PATCH 234/331] Remove twitter's HDFS proxied user patch, use oss version --- .../hive/BackgroundHiveSplitLoader.java | 34 +----------- .../presto/hive/HiveClientConfig.java | 15 ------ .../presto/hive/HivePageSourceProvider.java | 21 -------- .../presto/hive/HiveSessionProperties.java | 11 ---- .../presto/twitter/hive/util/UgiUtils.java | 52 ------------------- .../presto/hive/TestHiveClientConfig.java | 3 -- 6 files changed, 1 insertion(+), 135 deletions(-) delete mode 100644 presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/UgiUtils.java 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 6b6d9e8842a2..58a748b5f897 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 @@ -26,7 +26,6 @@ import com.facebook.presto.spi.StandardErrorCode; import com.facebook.presto.spi.predicate.Domain; import com.facebook.presto.spi.predicate.TupleDomain; -import com.facebook.presto.twitter.hive.util.UgiUtils; import com.google.common.base.Throwables; import com.google.common.collect.AbstractIterator; import com.google.common.collect.ImmutableList; @@ -48,14 +47,12 @@ import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.TextInputFormat; -import org.apache.hadoop.security.UserGroupInformation; import java.io.BufferedReader; import java.io.IOException; import java.io.InputStreamReader; import java.lang.annotation.Annotation; import java.nio.charset.StandardCharsets; -import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.Arrays; import java.util.Deque; @@ -174,15 +171,8 @@ public BackgroundHiveSplitLoader( public void start(HiveSplitSource splitSource) { this.hiveSplitSource = splitSource; - - UserGroupInformation ugi = null; - - if (HiveSessionProperties.getReadAsQueryUser(session)) { - ugi = UgiUtils.getUgi(session.getUser()); - } - for (int i = 0; i < maxPartitionBatchSize; i++) { - ResumableTasks.submit(executor, new HiveSplitLoaderTask(ugi)); + ResumableTasks.submit(executor, new HiveSplitLoaderTask()); } } @@ -195,30 +185,8 @@ public void stop() private class HiveSplitLoaderTask implements ResumableTask { - private UserGroupInformation ugi; - - public HiveSplitLoaderTask(UserGroupInformation ugi) - { - this.ugi = ugi; - } - @Override public TaskStatus process() - { - if (ugi != null) { - try { - return ugi.doAs((PrivilegedExceptionAction) this::doProcess); - } - catch (IOException | InterruptedException e) { - throw new RuntimeException("Could not runAs " + session.getUser(), e); - } - } - else { - return doProcess(); - } - } - - private TaskStatus doProcess() { while (true) { if (stopped) { 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 12fe2f7273f9..b6be50730e7c 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 @@ -60,8 +60,6 @@ public class HiveClientConfig private boolean allowCorruptWritesForTesting; - private boolean readAsQueryUser = false; - private Duration metastoreCacheTtl = new Duration(0, TimeUnit.SECONDS); private Duration metastoreRefreshInterval = new Duration(0, TimeUnit.SECONDS); private long metastoreCacheMaximumSize = 10000; @@ -283,19 +281,6 @@ public HiveClientConfig setAllowCorruptWritesForTesting(boolean allowCorruptWrit return this; } - public boolean getReadAsQueryUser() - { - return readAsQueryUser; - } - - @Config("hive.read-as-query-user") - @ConfigDescription("When querying hive read data as the user submitting the query instead of as the presto daemon user") - public HiveClientConfig setReadAsQueryUser(boolean readAsQueryUser) - { - this.readAsQueryUser = readAsQueryUser; - return this; - } - @NotNull public Duration getMetastoreCacheTtl() { 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 2256553f6039..27a3bb18774c 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 @@ -25,18 +25,14 @@ import com.facebook.presto.spi.predicate.TupleDomain; import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.TypeManager; -import com.facebook.presto.twitter.hive.util.UgiUtils; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.security.UserGroupInformation; import org.joda.time.DateTimeZone; import javax.inject.Inject; -import java.io.IOException; -import java.security.PrivilegedExceptionAction; import java.util.List; import java.util.Map; import java.util.Optional; @@ -81,23 +77,6 @@ public HivePageSourceProvider( @Override public ConnectorPageSource createPageSource(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorSplit split, List columns) - { - if (HiveSessionProperties.getReadAsQueryUser(session)) { - UserGroupInformation ugi = UgiUtils.getUgi(session.getUser()); - try { - return ugi.doAs((PrivilegedExceptionAction) () -> - doCreatePageSource(session, split, columns)); - } - catch (IOException | InterruptedException e) { - throw new RuntimeException("Could not runAs " + session.getUser(), e); - } - } - else { - return doCreatePageSource(session, split, columns); - } - } - - private ConnectorPageSource doCreatePageSource(ConnectorSession session, ConnectorSplit split, List columns) { List hiveColumns = columns.stream() .map(HiveColumnHandle.class::cast) 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 939399713ae7..92892e161190 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 @@ -39,7 +39,6 @@ public final class HiveSessionProperties private static final String ORC_OPTIMIZED_WRITER_VALIDATE = "orc_optimized_writer_validate"; 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"; private static final String MAX_SPLIT_SIZE = "max_split_size"; private static final String MAX_INITIAL_SPLIT_SIZE = "max_initial_split_size"; public static final String RCFILE_OPTIMIZED_WRITER_ENABLED = "rcfile_optimized_writer_enabled"; @@ -62,11 +61,6 @@ public HiveSessionProperties(HiveClientConfig config) "Only schedule splits on workers colocated with data node", config.isForceLocalScheduling(), false), - booleanSessionProperty( - READ_AS_QUERY_USER, - "Query reads happen as the user submitting the query", - config.getReadAsQueryUser(), - true), booleanSessionProperty( ORC_BLOOM_FILTERS_ENABLED, "ORC: Enable bloom filters for predicate pushdown", @@ -209,11 +203,6 @@ public static boolean isParquetPredicatePushdownEnabled(ConnectorSession session return session.getProperty(PARQUET_PREDICATE_PUSHDOWN_ENABLED, Boolean.class); } - public static boolean getReadAsQueryUser(ConnectorSession session) - { - return session.getProperty(READ_AS_QUERY_USER, Boolean.class); - } - public static DataSize getMaxSplitSize(ConnectorSession session) { return session.getProperty(MAX_SPLIT_SIZE, DataSize.class); diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/UgiUtils.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/UgiUtils.java deleted file mode 100644 index 6d540bbe3c5a..000000000000 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/UgiUtils.java +++ /dev/null @@ -1,52 +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.twitter.hive.util; - -import org.apache.hadoop.security.UserGroupInformation; - -import java.io.IOException; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -/** - * Utility class to handle creating and caching the UserGroupInformation object. - */ -public class UgiUtils -{ - private UgiUtils() {} - - // Every instance of a UserGroupInformation object for a given user has a unique hashcode, due - // to the hashCode() impl. If we don't cache the UGI per-user here, there will be a memory leak - // in the PrestoFileSystemCache. - private static final Map UGI_CACHE = new ConcurrentHashMap<>(); - - public static UserGroupInformation getUgi(String user) - { - UserGroupInformation ugi = UGI_CACHE.get(user); - - if (ugi == null) { - // Configure hadoop to allow presto daemon user to impersonate all presto users - // See https://hadoop.apache.org/docs/r2.4.1/hadoop-project-dist/hadoop-common/Superusers.html - try { - ugi = UserGroupInformation.createProxyUser(user, UserGroupInformation.getLoginUser()); - } - catch (IOException e) { - throw new RuntimeException("Could not get login user from UserGroupInformation", e); - } - UGI_CACHE.put(user, ugi); - } - - return ugi; - } -} 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 7c2495219926..d1ca0ca5dff2 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 @@ -67,7 +67,6 @@ public void testDefaults() .setHiveStorageFormat(HiveStorageFormat.RCBINARY) .setHiveCompressionCodec(HiveCompressionCodec.GZIP) .setRespectTableFormat(true) - .setReadAsQueryUser(false) .setImmutablePartitions(false) .setMaxPartitionsPerWriter(100) .setWriteValidationThreads(16) @@ -137,7 +136,6 @@ public void testExplicitPropertyMappings() .put("hive.max-concurrent-file-renames", "100") .put("hive.assume-canonical-partition-keys", "true") .put("hive.parquet.use-column-names", "true") - .put("hive.read-as-query-user", "true") .put("hive.orc.use-column-names", "true") .put("hive.parquet-predicate-pushdown.enabled", "true") .put("hive.parquet-optimized-reader.enabled", "true") @@ -191,7 +189,6 @@ public void testExplicitPropertyMappings() .setVerifyChecksum(false) .setResourceConfigFiles(ImmutableList.of("/foo.xml", "/bar.xml")) .setHiveStorageFormat(HiveStorageFormat.SEQUENCEFILE) - .setReadAsQueryUser(true) .setHiveCompressionCodec(HiveCompressionCodec.NONE) .setRespectTableFormat(false) .setImmutablePartitions(true) From 359c77cfaecc587ac7a6ef6b47c1cea095fcdd5f Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Sun, 8 Oct 2017 21:40:25 -0700 Subject: [PATCH 235/331] Use a non-default executer in impersonation test --- .../presto/tests/ImpersonationTests.java | 29 +++++++++++++------ 1 file changed, 20 insertions(+), 9 deletions(-) diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/ImpersonationTests.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/ImpersonationTests.java index 63172c301104..2c0a07be2447 100644 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/ImpersonationTests.java +++ b/presto-product-tests/src/main/java/com/facebook/presto/tests/ImpersonationTests.java @@ -15,16 +15,18 @@ import com.google.inject.Inject; import com.google.inject.name.Named; +import com.teradata.tempto.BeforeTestWithContext; import com.teradata.tempto.ProductTest; import com.teradata.tempto.Requires; import com.teradata.tempto.fulfillment.table.hive.tpch.ImmutableTpchTablesRequirements.ImmutableNationTable; import com.teradata.tempto.hadoop.hdfs.HdfsClient; +import com.teradata.tempto.query.QueryExecutor; import org.testng.annotations.Test; import static com.facebook.presto.tests.TestGroups.HDFS_IMPERSONATION; import static com.facebook.presto.tests.TestGroups.HDFS_NO_IMPERSONATION; import static com.facebook.presto.tests.TestGroups.PROFILE_SPECIFIC_TESTS; -import static com.teradata.tempto.query.QueryExecutor.query; +import static com.facebook.presto.tests.utils.QueryExecutors.connectToPresto; import static java.lang.String.format; import static org.testng.Assert.assertEquals; @@ -32,12 +34,14 @@ public class ImpersonationTests extends ProductTest { + private QueryExecutor aliceExecutor; + @Inject private HdfsClient hdfsClient; @Inject - @Named("databases.presto.jdbc_user") - private String prestoJdbcUser; + @Named("databases.alice@presto.jdbc_user") + private String aliceJdbcUser; // The value for configuredHdfsUser is profile dependent // For non-Kerberos environments this variable will be equal to -DHADOOP_USER_NAME as set in jvm.config @@ -50,12 +54,19 @@ public class ImpersonationTests @Named("databases.hive.warehouse_directory_path") private String warehouseDirectoryPath; + @BeforeTestWithContext + public void setup() + { + aliceExecutor = connectToPresto("alice@presto"); + QueryExecutor.query(format("GRANT SELECT ON NATION TO %s", aliceJdbcUser)); + } + @Test(groups = {HDFS_NO_IMPERSONATION, PROFILE_SPECIFIC_TESTS}) public void testHdfsImpersonationDisabled() throws Exception { String tableName = "check_hdfs_impersonation_disabled"; - checkTableOwner(tableName, configuredHdfsUser); + checkTableOwner(tableName, configuredHdfsUser, aliceExecutor); } @Test(groups = {HDFS_IMPERSONATION, PROFILE_SPECIFIC_TESTS}) @@ -63,7 +74,7 @@ public void testHdfsImpersonationEnabled() throws Exception { String tableName = "check_hdfs_impersonation_enabled"; - checkTableOwner(tableName, prestoJdbcUser); + checkTableOwner(tableName, aliceJdbcUser, aliceExecutor); } private String getTableLocation(String tableName) @@ -71,13 +82,13 @@ private String getTableLocation(String tableName) return warehouseDirectoryPath + '/' + tableName; } - private void checkTableOwner(String tableName, String expectedOwner) + private void checkTableOwner(String tableName, String expectedOwner, QueryExecutor executor) { - query(format("DROP TABLE IF EXISTS %s", tableName)); - query(format("CREATE TABLE %s AS SELECT * FROM NATION", tableName)); + executor.executeQuery(format("DROP TABLE IF EXISTS %s", tableName)); + executor.executeQuery(format("CREATE TABLE %s AS SELECT * FROM NATION", tableName)); String tableLocation = getTableLocation(tableName); String owner = hdfsClient.getOwner(tableLocation); assertEquals(owner, expectedOwner); - query(format("DROP TABLE IF EXISTS %s", tableName)); + executor.executeQuery(format("DROP TABLE IF EXISTS %s", tableName)); } } From dc17415fc4ca6d8782b10770e147525860609476 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Sun, 8 Oct 2017 21:33:50 -0700 Subject: [PATCH 236/331] Add simple hdfs impersonation test in Travis CI --- .travis.yml | 10 ++++++++++ .../presto/tests/hive/TestHiveStorageFormats.java | 7 +++++-- 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/.travis.yml b/.travis.yml index 782790842e1b..ecaef70c6f2f 100644 --- a/.travis.yml +++ b/.travis.yml @@ -92,6 +92,16 @@ script: presto-product-tests/bin/run_on_docker.sh \ multinode -x quarantine,big_query,storage_formats,profile_specific_tests,tpcds,cli,hive_connector fi + - | + if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then + presto-product-tests/bin/run_on_docker.sh \ + singlenode -g hdfs_no_impersonation + fi + - | + if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then + presto-product-tests/bin/run_on_docker.sh \ + singlenode-hdfs-impersonation -g storage_formats,cli,hdfs_impersonation + fi - | if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then presto-product-tests/bin/run_on_docker.sh \ diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java index b0e373ee1d98..a14eb5ad2959 100644 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java +++ b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveStorageFormats.java @@ -47,11 +47,14 @@ public static Object[][] storageFormats() { return new StorageFormat[][] { {storageFormat("ORC")}, + {storageFormat("ORC", ImmutableMap.of("hive.orc_optimized_writer_enabled", "true", "hive.orc_optimized_writer_validate", "true"))}, {storageFormat("DWRF")}, {storageFormat("PARQUET")}, {storageFormat("PARQUET", ImmutableMap.of("hive.parquet_optimized_reader_enabled", "true"))}, - {storageFormat("RCBINARY")}, - {storageFormat("RCTEXT")}, + {storageFormat("RCBINARY", ImmutableMap.of("hive.rcfile_optimized_writer_enabled", "false", "hive.rcfile_optimized_writer_validate", "false"))}, + {storageFormat("RCBINARY", ImmutableMap.of("hive.rcfile_optimized_writer_enabled", "true", "hive.rcfile_optimized_writer_validate", "true"))}, + {storageFormat("RCTEXT", ImmutableMap.of("hive.rcfile_optimized_writer_enabled", "false", "hive.rcfile_optimized_writer_validate", "false"))}, + {storageFormat("RCTEXT", ImmutableMap.of("hive.rcfile_optimized_writer_enabled", "true", "hive.rcfile_optimized_writer_validate", "true"))}, {storageFormat("SEQUENCEFILE")}, {storageFormat("TEXTFILE")} }; From eabf99c31ce279708d359e7f41890065971ac86e Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Sat, 7 Oct 2017 03:03:37 -0700 Subject: [PATCH 237/331] Use loginUser instead of currentUser in SimpleHadoopAuthentication --- .../presto/hive/authentication/SimpleHadoopAuthentication.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/SimpleHadoopAuthentication.java b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/SimpleHadoopAuthentication.java index 663d852e4399..0aced6734483 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/authentication/SimpleHadoopAuthentication.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/authentication/SimpleHadoopAuthentication.java @@ -25,7 +25,7 @@ public class SimpleHadoopAuthentication public UserGroupInformation getUserGroupInformation() { try { - return UserGroupInformation.getCurrentUser(); + return UserGroupInformation.getLoginUser(); } catch (IOException e) { throw Throwables.propagate(e); From 2f1b889c682f00948926c793866ab7f9af23abe2 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 4 Oct 2017 15:27:42 -0700 Subject: [PATCH 238/331] Added Parquet Structural Type Evolution Test --- .../presto/hive/AbstractTestHiveClient.java | 19 ++++++++++++++++++- .../src/test/sql/create-test-hive13.sql | 7 ++++++- 2 files changed, 24 insertions(+), 2 deletions(-) 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 1f03b6a96412..bae8b1238f21 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 @@ -73,6 +73,7 @@ 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.RowType; import com.facebook.presto.spi.type.SqlDate; import com.facebook.presto.spi.type.SqlTimestamp; import com.facebook.presto.spi.type.SqlVarbinary; @@ -2846,6 +2847,22 @@ else if (rowNumber % 39 == 1) { } } + // STRUCT + index = columnIndex.get("t_struct"); + if (index != null) { + if ((rowNumber % 31) == 0) { + assertNull(row.getField(index)); + } + else { + assertTrue(row.getField(index) instanceof List); + List values = (List) row.getField(index); + assertEquals(values.size(), 3); + assertEquals(values.get(0), "test abc"); + assertEquals(values.get(1), 0.1); + assertNull(values.get(2)); + } + } + // MAP>> index = columnIndex.get("t_complex"); if (index != null) { @@ -3080,7 +3097,7 @@ else if (TIMESTAMP.equals(column.getType())) { else if (DATE.equals(column.getType())) { assertInstanceOf(value, SqlDate.class); } - else if (column.getType() instanceof ArrayType) { + else if (column.getType() instanceof ArrayType || column.getType() instanceof RowType) { assertInstanceOf(value, List.class); } else if (column.getType() instanceof MapType) { diff --git a/presto-hive/src/test/sql/create-test-hive13.sql b/presto-hive/src/test/sql/create-test-hive13.sql index 0d63bf67ce1f..e0f9237e8c6c 100644 --- a/presto-hive/src/test/sql/create-test-hive13.sql +++ b/presto-hive/src/test/sql/create-test-hive13.sql @@ -82,7 +82,6 @@ SELECT * FROM presto_test_types_textfile ; --- Parquet fails when trying to use complex nested types. -- Parquet is missing TIMESTAMP and BINARY. CREATE TABLE presto_test_types_parquet ( t_string STRING @@ -99,11 +98,14 @@ CREATE TABLE presto_test_types_parquet ( , t_map MAP , t_array_string ARRAY , t_array_struct ARRAY> +, t_struct STRUCT ) +PARTITIONED BY (dummy INT) STORED AS PARQUET ; INSERT INTO TABLE presto_test_types_parquet +PARTITION (dummy=0) SELECT t_string , t_varchar @@ -119,9 +121,12 @@ SELECT , t_map , t_array_string , t_array_struct +, t_array_struct[0] FROM presto_test_types_textfile ; +ALTER TABLE presto_test_types_parquet +CHANGE COLUMN t_struct t_struct STRUCT; ALTER TABLE presto_test_types_textfile ADD COLUMNS (new_column INT); ALTER TABLE presto_test_types_sequencefile ADD COLUMNS (new_column INT); From b5f6cd75406abcd505e4ada0b0b93579c42db1aa Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 4 Oct 2017 17:09:39 -0700 Subject: [PATCH 239/331] Allow structural coercion in HiveCoercionPolicy --- .../presto/hive/HiveCoercionPolicy.java | 53 ++++++++++++++++++- 1 file changed, 52 insertions(+), 1 deletion(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveCoercionPolicy.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveCoercionPolicy.java index ed9a5a505ad9..7224b35cc791 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveCoercionPolicy.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveCoercionPolicy.java @@ -16,15 +16,23 @@ import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.TypeManager; import com.facebook.presto.spi.type.VarcharType; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category; +import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; import javax.inject.Inject; +import java.util.List; +import java.util.stream.Collectors; + import static com.facebook.presto.hive.HiveType.HIVE_BYTE; import static com.facebook.presto.hive.HiveType.HIVE_DOUBLE; import static com.facebook.presto.hive.HiveType.HIVE_FLOAT; import static com.facebook.presto.hive.HiveType.HIVE_INT; import static com.facebook.presto.hive.HiveType.HIVE_LONG; import static com.facebook.presto.hive.HiveType.HIVE_SHORT; + import static java.util.Objects.requireNonNull; public class HiveCoercionPolicy @@ -62,6 +70,49 @@ public boolean canCoerce(HiveType fromHiveType, HiveType toHiveType) return toHiveType.equals(HIVE_DOUBLE); } - return false; + return canCoerceForList(fromHiveType, toHiveType) || canCoerceForMap(fromHiveType, toHiveType) || canCoerceForStruct(fromHiveType, toHiveType); + } + + private boolean canCoerceForMap(HiveType fromHiveType, HiveType toHiveType) + { + if (!fromHiveType.getCategory().equals(Category.MAP) || !toHiveType.getCategory().equals(Category.MAP)) { + return false; + } + HiveType fromKeyType = HiveType.valueOf(((MapTypeInfo) fromHiveType.getTypeInfo()).getMapKeyTypeInfo().getTypeName()); + HiveType fromValueType = HiveType.valueOf(((MapTypeInfo) fromHiveType.getTypeInfo()).getMapValueTypeInfo().getTypeName()); + HiveType toKeyType = HiveType.valueOf(((MapTypeInfo) toHiveType.getTypeInfo()).getMapKeyTypeInfo().getTypeName()); + HiveType toValueType = HiveType.valueOf(((MapTypeInfo) toHiveType.getTypeInfo()).getMapValueTypeInfo().getTypeName()); + return (fromKeyType.equals(toKeyType) || canCoerce(fromKeyType, toKeyType)) && (fromValueType.equals(toValueType) || canCoerce(fromValueType, toValueType)); + } + + private boolean canCoerceForList(HiveType fromHiveType, HiveType toHiveType) + { + if (!fromHiveType.getCategory().equals(Category.LIST) || !toHiveType.getCategory().equals(Category.LIST)) { + return false; + } + HiveType fromElementType = HiveType.valueOf(((ListTypeInfo) fromHiveType.getTypeInfo()).getListElementTypeInfo().getTypeName()); + HiveType toElementType = HiveType.valueOf(((ListTypeInfo) toHiveType.getTypeInfo()).getListElementTypeInfo().getTypeName()); + return fromElementType.equals(toElementType) || canCoerce(fromElementType, toElementType); + } + + private boolean canCoerceForStruct(HiveType fromHiveType, HiveType toHiveType) + { + if (!fromHiveType.getCategory().equals(Category.STRUCT) || !toHiveType.getCategory().equals(Category.STRUCT)) { + return false; + } + List fromFieldTypes = getAllStructFieldTypeInfos(fromHiveType); + List toFieldTypes = getAllStructFieldTypeInfos(toHiveType); + for (int i = 0; i < Math.min(fromFieldTypes.size(), toFieldTypes.size()); i++) { + if (!fromFieldTypes.get(i).equals(toFieldTypes.get(i)) && !canCoerce(fromFieldTypes.get(i), toFieldTypes.get(i))) { + return false; + } + } + return true; + } + + private List getAllStructFieldTypeInfos(HiveType hiveType) + { + return ((StructTypeInfo) hiveType.getTypeInfo()).getAllStructFieldTypeInfos() + .stream().map(typeInfo -> HiveType.valueOf(typeInfo.getTypeName())).collect(Collectors.toList()); } } From e868f5668e48f0b073185ff9dc109e381278c185 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 10 Oct 2017 09:00:21 -0700 Subject: [PATCH 240/331] Add support to coerce structural type --- .../presto/hive/HiveCoercionRecordCursor.java | 306 ++++++++++++++++++ .../facebook/presto/hive/HivePageSource.java | 209 ++++++++++++ .../presto/hive/AbstractTestHiveClient.java | 87 ++++- 3 files changed, 597 insertions(+), 5 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveCoercionRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveCoercionRecordCursor.java index 442be5042d91..7f3f77dd1a96 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveCoercionRecordCursor.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveCoercionRecordCursor.java @@ -16,14 +16,23 @@ import com.facebook.presto.hive.HivePageSourceProvider.ColumnMapping; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.RecordCursor; +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.Type; import com.facebook.presto.spi.type.TypeManager; import com.facebook.presto.spi.type.VarcharType; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import io.airlift.slice.Slice; +import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; +import java.util.Arrays; import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; import static com.facebook.presto.hive.HiveType.HIVE_BYTE; import static com.facebook.presto.hive.HiveType.HIVE_DOUBLE; @@ -288,6 +297,16 @@ else if (fromHiveType.equals(HIVE_INT) && toHiveType.equals(HIVE_LONG)) { else if (fromHiveType.equals(HIVE_FLOAT) && toHiveType.equals(HIVE_DOUBLE)) { return new FloatToDoubleCoercer(); } + else if (HiveUtil.isArrayType(fromType) && HiveUtil.isArrayType(toType)) { + return new ListToListCoercer(typeManager, fromHiveType, toHiveType); + } + else if (HiveUtil.isMapType(fromType) && HiveUtil.isMapType(toType)) { + return new MapToMapCoercer(typeManager, fromHiveType, toHiveType); + } + else if (HiveUtil.isRowType(fromType) && HiveUtil.isRowType(toType)) { + return new StructToStructCoercer(typeManager, fromHiveType, toHiveType); + } + throw new PrestoException(NOT_SUPPORTED, format("Unsupported coercion from %s to %s", fromHiveType, toHiveType)); } @@ -367,4 +386,291 @@ public void coerce(RecordCursor delegate, int field) } } } + + private static class ListToListCoercer + extends Coercer + { + private final TypeManager typeManager; + private final HiveType fromHiveType; + private final HiveType toHiveType; + private final HiveType fromElementHiveType; + private final HiveType toElementHiveType; + private final Coercer elementCoercer; + + public ListToListCoercer(TypeManager typeManager, HiveType fromHiveType, HiveType toHiveType) + { + this.typeManager = requireNonNull(typeManager, "typeManage is null"); + this.fromHiveType = requireNonNull(fromHiveType, "fromHiveType is null"); + this.toHiveType = requireNonNull(toHiveType, "toHiveType is null"); + this.fromElementHiveType = HiveType.valueOf(((ListTypeInfo) fromHiveType.getTypeInfo()).getListElementTypeInfo().getTypeName()); + this.toElementHiveType = HiveType.valueOf(((ListTypeInfo) toHiveType.getTypeInfo()).getListElementTypeInfo().getTypeName()); + this.elementCoercer = fromElementHiveType.equals(toElementHiveType) ? null : createCoercer(typeManager, fromElementHiveType, toElementHiveType); + } + + @Override + public void coerce(RecordCursor delegate, int field) + { + if (delegate.isNull(field)) { + setIsNull(true); + return; + } + Block block = (Block) delegate.getObject(field); + BlockBuilder builder = toHiveType.getType(typeManager).createBlockBuilder(new BlockBuilderStatus(), 1); + BlockBuilder listBuilder = builder.beginBlockEntry(); + for (int i = 0; i < block.getPositionCount(); i++) { + if (block.isNull(i)) { + listBuilder.appendNull(); + } + else if (elementCoercer == null) { + block.writePositionTo(i, listBuilder); + listBuilder.closeEntry(); + } + else { + rewriteBlock(fromElementHiveType, toElementHiveType, block, i, listBuilder, elementCoercer, typeManager, field); + } + } + builder.closeEntry(); + setObject(builder.build().getObject(0, Block.class)); + } + } + + private static class MapToMapCoercer + extends Coercer + { + private final TypeManager typeManager; + private final HiveType fromHiveType; + private final HiveType toHiveType; + private final HiveType fromKeyHiveType; + private final HiveType toKeyHiveType; + private final HiveType fromValueHiveType; + private final HiveType toValueHiveType; + private final Coercer keyCoercer; + private final Coercer valueCoercer; + + public MapToMapCoercer(TypeManager typeManager, HiveType fromHiveType, HiveType toHiveType) + { + this.typeManager = requireNonNull(typeManager, "typeManage is null"); + this.fromHiveType = requireNonNull(fromHiveType, "fromHiveType is null"); + this.toHiveType = requireNonNull(toHiveType, "toHiveType is null"); + this.fromKeyHiveType = HiveType.valueOf(((MapTypeInfo) fromHiveType.getTypeInfo()).getMapKeyTypeInfo().getTypeName()); + this.fromValueHiveType = HiveType.valueOf(((MapTypeInfo) fromHiveType.getTypeInfo()).getMapValueTypeInfo().getTypeName()); + this.toKeyHiveType = HiveType.valueOf(((MapTypeInfo) toHiveType.getTypeInfo()).getMapKeyTypeInfo().getTypeName()); + this.toValueHiveType = HiveType.valueOf(((MapTypeInfo) toHiveType.getTypeInfo()).getMapValueTypeInfo().getTypeName()); + this.keyCoercer = fromKeyHiveType.equals(toKeyHiveType) ? null : createCoercer(typeManager, fromKeyHiveType, toKeyHiveType); + this.valueCoercer = fromValueHiveType.equals(toValueHiveType) ? null : createCoercer(typeManager, fromValueHiveType, toValueHiveType); + } + + @Override + public void coerce(RecordCursor delegate, int field) + { + if (delegate.isNull(field)) { + setIsNull(true); + return; + } + Block block = (Block) delegate.getObject(field); + BlockBuilder builder = toHiveType.getType(typeManager).createBlockBuilder(new BlockBuilderStatus(), 1); + BlockBuilder mapBuilder = builder.beginBlockEntry(); + for (int i = 0; i < block.getPositionCount(); i += 2) { + if (block.isNull(i)) { + mapBuilder.appendNull(); + } + else if (keyCoercer == null) { + block.writePositionTo(i, mapBuilder); + mapBuilder.closeEntry(); + } + else { + rewriteBlock(fromKeyHiveType, toKeyHiveType, block.getSingleValueBlock(i), 0, mapBuilder, keyCoercer, typeManager, field); + } + if (block.isNull(i + 1)) { + mapBuilder.appendNull(); + } + if (valueCoercer == null) { + block.writePositionTo(i + 1, mapBuilder); + mapBuilder.closeEntry(); + } + else { + rewriteBlock(fromValueHiveType, toValueHiveType, block.getSingleValueBlock(i + 1), 0, mapBuilder, valueCoercer, typeManager, field); + } + } + builder.closeEntry(); + setObject(builder.build().getObject(0, Block.class)); + } + } + + private static class StructToStructCoercer + extends Coercer + { + private final TypeManager typeManager; + private final HiveType fromHiveType; + private final HiveType toHiveType; + private final List fromFieldTypes; + private final List toFieldTypes; + private final Coercer[] coercers; + + public StructToStructCoercer(TypeManager typeManager, HiveType fromHiveType, HiveType toHiveType) + { + this.typeManager = requireNonNull(typeManager, "typeManage is null"); + this.fromHiveType = requireNonNull(fromHiveType, "fromHiveType is null"); + this.toHiveType = requireNonNull(toHiveType, "toHiveType is null"); + this.fromFieldTypes = getAllStructFieldTypeInfos(fromHiveType); + this.toFieldTypes = getAllStructFieldTypeInfos(toHiveType); + this.coercers = new Coercer[toFieldTypes.size()]; + Arrays.fill(this.coercers, null); + for (int i = 0; i < Math.min(fromFieldTypes.size(), toFieldTypes.size()); i++) { + if (!fromFieldTypes.get(i).equals(toFieldTypes.get(i))) { + coercers[i] = createCoercer(typeManager, fromFieldTypes.get(i), toFieldTypes.get(i)); + } + } + } + + @Override + public void coerce(RecordCursor delegate, int field) + { + if (delegate.isNull(field)) { + setIsNull(true); + return; + } + Block block = (Block) delegate.getObject(field); + BlockBuilder builder = toHiveType.getType(typeManager).createBlockBuilder(new BlockBuilderStatus(), 1); + BlockBuilder rowBuilder = builder.beginBlockEntry(); + for (int i = 0; i < toFieldTypes.size(); i++) { + if (i >= fromFieldTypes.size() || block.isNull(i)) { + rowBuilder.appendNull(); + } + else if (coercers[i] == null) { + block.writePositionTo(i, rowBuilder); + rowBuilder.closeEntry(); + } + else { + rewriteBlock(fromFieldTypes.get(i), toFieldTypes.get(i), block, i, rowBuilder, coercers[i], typeManager, field); + } + } + builder.closeEntry(); + setObject(builder.build().getObject(0, Block.class)); + } + } + + private static void rewriteBlock( + HiveType fromFieldHiveType, + HiveType toFieldHiveType, + Block block, + int position, + BlockBuilder builder, + Coercer coercer, + TypeManager typeManager, + int field) + { + Type fromFieldType = fromFieldHiveType.getType(typeManager); + Type toFieldType = toFieldHiveType.getType(typeManager); + Object value = null; + if (fromFieldHiveType.equals(HIVE_BYTE) || fromFieldHiveType.equals(HIVE_SHORT) || fromFieldHiveType.equals(HIVE_INT) || fromFieldHiveType.equals(HIVE_LONG) || fromFieldHiveType.equals(HIVE_FLOAT)) { + value = fromFieldType.getLong(block, position); + } + else if (fromFieldType instanceof VarcharType) { + value = fromFieldType.getSlice(block, position); + } + else if (HiveUtil.isStructuralType(fromFieldHiveType)) { + value = fromFieldType.getObject(block, position); + } + coercer.reset(); + RecordCursor bridgingRecordCursor = createBridgingRecordCursor(value, typeManager, fromFieldHiveType); + if (coercer.isNull(bridgingRecordCursor, field)) { + builder.appendNull(); + } + else if (toFieldHiveType.equals(HIVE_BYTE) || toFieldHiveType.equals(HIVE_SHORT) || toFieldHiveType.equals(HIVE_INT) || toFieldHiveType.equals(HIVE_LONG) || toFieldHiveType.equals(HIVE_FLOAT)) { + toFieldType.writeLong(builder, coercer.getLong(bridgingRecordCursor, field)); + } + else if (toFieldHiveType.equals(HIVE_DOUBLE)) { + toFieldType.writeDouble(builder, coercer.getDouble(bridgingRecordCursor, field)); + } + else if (toFieldType instanceof VarcharType) { + toFieldType.writeSlice(builder, coercer.getSlice(bridgingRecordCursor, field)); + } + else if (HiveUtil.isStructuralType(toFieldHiveType)) { + toFieldType.writeObject(builder, coercer.getObject(bridgingRecordCursor, field)); + } + else { + throw new PrestoException(NOT_SUPPORTED, format("Unsupported coercion from %s to %s", fromFieldHiveType, toFieldHiveType)); + } + coercer.reset(); + } + + private static RecordCursor createBridgingRecordCursor( + Object value, + TypeManager typeManager, + HiveType hiveType) + { + return new RecordCursor() { + @Override + public long getCompletedBytes() + { + return 0; + } + + @Override + public long getReadTimeNanos() + { + return 0; + } + + @Override + public Type getType(int field) + { + return hiveType.getType(typeManager); + } + + @Override + public boolean advanceNextPosition() + { + return true; + } + + @Override + public boolean getBoolean(int field) + { + return (Boolean) value; + } + + @Override + public long getLong(int field) + { + return (Long) value; + } + + @Override + public double getDouble(int field) + { + return (Double) value; + } + + @Override + public Slice getSlice(int field) + { + return (Slice) value; + } + + @Override + public Object getObject(int field) + { + return value; + } + + @Override + public boolean isNull(int field) + { + return Objects.isNull(value); + } + + @Override + public void close() + { + } + }; + } + + private static List getAllStructFieldTypeInfos(HiveType hiveType) + { + return ((StructTypeInfo) hiveType.getTypeInfo()).getAllStructFieldTypeInfos() + .stream().map(typeInfo -> HiveType.valueOf(typeInfo.getTypeName())).collect(Collectors.toList()); + } } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSource.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSource.java index 4fcebf1798ec..6237f949b57d 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSource.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSource.java @@ -28,11 +28,16 @@ import com.facebook.presto.spi.type.TypeManager; import com.facebook.presto.spi.type.VarcharType; import com.google.common.base.Throwables; +import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; import org.joda.time.DateTimeZone; import java.io.IOException; +import java.util.Arrays; import java.util.List; import java.util.function.Function; +import java.util.stream.Collectors; import static com.facebook.presto.hive.HiveErrorCode.HIVE_CURSOR_ERROR; import static com.facebook.presto.hive.HiveType.HIVE_BYTE; @@ -287,6 +292,15 @@ else if (fromHiveType.equals(HIVE_INT) && toHiveType.equals(HIVE_LONG)) { else if (fromHiveType.equals(HIVE_FLOAT) && toHiveType.equals(HIVE_DOUBLE)) { return new FloatToDoubleCoercer(); } + else if (HiveUtil.isArrayType(fromType) && HiveUtil.isArrayType(toType)) { + return new ListToListCoercer(typeManager, fromHiveType, toHiveType); + } + else if (HiveUtil.isMapType(fromType) && HiveUtil.isMapType(toType)) { + return new MapToMapCoercer(typeManager, fromHiveType, toHiveType); + } + else if (HiveUtil.isRowType(fromType) && HiveUtil.isRowType(toType)) { + return new StructToStructCoercer(typeManager, fromHiveType, toHiveType); + } throw new PrestoException(NOT_SUPPORTED, format("Unsupported coercion from %s to %s", fromHiveType, toHiveType)); } @@ -424,6 +438,195 @@ public Block apply(Block block) } } + private static class ListToListCoercer + implements Function + { + private final TypeManager typeManager; + private final HiveType fromHiveType; + private final HiveType toHiveType; + private final Function elementCoercer; + + public ListToListCoercer(TypeManager typeManager, HiveType fromHiveType, HiveType toHiveType) + { + this.typeManager = requireNonNull(typeManager, "typeManage is null"); + this.fromHiveType = requireNonNull(fromHiveType, "fromHiveType is null"); + this.toHiveType = requireNonNull(toHiveType, "toHiveType is null"); + HiveType fromElementHiveType = HiveType.valueOf(((ListTypeInfo) fromHiveType.getTypeInfo()).getListElementTypeInfo().getTypeName()); + HiveType toElementHiveType = HiveType.valueOf(((ListTypeInfo) toHiveType.getTypeInfo()).getListElementTypeInfo().getTypeName()); + this.elementCoercer = fromElementHiveType.equals(toElementHiveType) ? null : createCoercer(typeManager, fromElementHiveType, toElementHiveType); + } + + @Override + public Block apply(Block block) + { + BlockBuilder blockBuilder = toHiveType.getType(typeManager).createBlockBuilder(new BlockBuilderStatus(), block.getPositionCount()); + for (int i = 0; i < block.getPositionCount(); i++) { + if (block.isNull(i)) { + blockBuilder.appendNull(); + continue; + } + Block singleArrayBlock = block.getObject(i, Block.class); + BlockBuilder singleArrayBuilder = blockBuilder.beginBlockEntry(); + for (int j = 0; j < singleArrayBlock.getPositionCount(); j++) { + if (singleArrayBlock.isNull(j)) { + singleArrayBuilder.appendNull(); + } + else if (elementCoercer == null) { + singleArrayBlock.writePositionTo(j, singleArrayBuilder); + singleArrayBuilder.closeEntry(); + } + else { + Block singleElementBlock = elementCoercer.apply(singleArrayBlock.getSingleValueBlock(j)); + if (singleElementBlock.isNull(0)) { + singleArrayBuilder.appendNull(); + } + else { + singleElementBlock.writePositionTo(0, singleArrayBuilder); + singleArrayBuilder.closeEntry(); + } + } + } + blockBuilder.closeEntry(); + } + return blockBuilder.build(); + } + } + + private static class MapToMapCoercer + implements Function + { + private final TypeManager typeManager; + private final HiveType fromHiveType; + private final HiveType toHiveType; + private final Function keyCoercer; + private final Function valueCoercer; + + public MapToMapCoercer(TypeManager typeManager, HiveType fromHiveType, HiveType toHiveType) + { + this.typeManager = requireNonNull(typeManager, "typeManage is null"); + this.fromHiveType = requireNonNull(fromHiveType, "fromHiveType is null"); + this.toHiveType = requireNonNull(toHiveType, "toHiveType is null"); + HiveType fromKeyHiveType = HiveType.valueOf(((MapTypeInfo) fromHiveType.getTypeInfo()).getMapKeyTypeInfo().getTypeName()); + HiveType fromValueHiveType = HiveType.valueOf(((MapTypeInfo) fromHiveType.getTypeInfo()).getMapValueTypeInfo().getTypeName()); + HiveType toKeyHiveType = HiveType.valueOf(((MapTypeInfo) toHiveType.getTypeInfo()).getMapKeyTypeInfo().getTypeName()); + HiveType toValueHiveType = HiveType.valueOf(((MapTypeInfo) toHiveType.getTypeInfo()).getMapValueTypeInfo().getTypeName()); + this.keyCoercer = fromKeyHiveType.equals(toKeyHiveType) ? null : createCoercer(typeManager, fromKeyHiveType, toKeyHiveType); + this.valueCoercer = fromValueHiveType.equals(toValueHiveType) ? null : createCoercer(typeManager, fromValueHiveType, toValueHiveType); + } + + @Override + public Block apply(Block block) + { + BlockBuilder blockBuilder = toHiveType.getType(typeManager).createBlockBuilder(new BlockBuilderStatus(), block.getPositionCount()); + for (int i = 0; i < block.getPositionCount(); i++) { + if (block.isNull(i)) { + blockBuilder.appendNull(); + continue; + } + Block singleMapBlock = block.getObject(i, Block.class); + BlockBuilder singleMapBuilder = blockBuilder.beginBlockEntry(); + if (singleMapBlock.isNull(0)) { + singleMapBuilder.appendNull(); + } + else if (keyCoercer == null) { + singleMapBlock.writePositionTo(0, singleMapBuilder); + singleMapBuilder.closeEntry(); + } + else { + Block singleKeyBlock = keyCoercer.apply(singleMapBlock.getSingleValueBlock(0)); + if (singleKeyBlock.isNull(0)) { + singleMapBuilder.appendNull(); + } + else { + singleKeyBlock.writePositionTo(0, singleMapBuilder); + singleMapBuilder.closeEntry(); + } + } + if (singleMapBlock.isNull(1)) { + singleMapBuilder.appendNull(); + } + else if (valueCoercer == null) { + singleMapBlock.writePositionTo(1, singleMapBuilder); + singleMapBuilder.closeEntry(); + } + else { + Block singleValueBlock = valueCoercer.apply(singleMapBlock.getSingleValueBlock(1)); + if (singleValueBlock.isNull(0)) { + singleMapBuilder.appendNull(); + } + else { + singleValueBlock.writePositionTo(0, singleMapBuilder); + singleMapBuilder.closeEntry(); + } + } + blockBuilder.closeEntry(); + } + return blockBuilder.build(); + } + } + + private static class StructToStructCoercer + implements Function + { + private final TypeManager typeManager; + private final HiveType fromHiveType; + private final HiveType toHiveType; + private final List fromFieldTypes; + private final List toFieldTypes; + private final Function[] coercers; + + public StructToStructCoercer(TypeManager typeManager, HiveType fromHiveType, HiveType toHiveType) + { + this.typeManager = requireNonNull(typeManager, "typeManage is null"); + this.fromHiveType = requireNonNull(fromHiveType, "fromHiveType is null"); + this.toHiveType = requireNonNull(toHiveType, "toHiveType is null"); + this.fromFieldTypes = getAllStructFieldTypeInfos(fromHiveType); + this.toFieldTypes = getAllStructFieldTypeInfos(toHiveType); + this.coercers = new Function[toFieldTypes.size()]; + Arrays.fill(this.coercers, null); + for (int i = 0; i < Math.min(fromFieldTypes.size(), toFieldTypes.size()); i++) { + if (!fromFieldTypes.get(i).equals(toFieldTypes.get(i))) { + coercers[i] = createCoercer(typeManager, fromFieldTypes.get(i), toFieldTypes.get(i)); + } + } + } + + @Override + public Block apply(Block block) + { + BlockBuilder blockBuilder = toHiveType.getType(typeManager).createBlockBuilder(new BlockBuilderStatus(), block.getPositionCount()); + for (int i = 0; i < block.getPositionCount(); i++) { + if (block.isNull(i)) { + blockBuilder.appendNull(); + continue; + } + Block singleRowBlock = block.getObject(i, Block.class); + BlockBuilder singleRowBuilder = blockBuilder.beginBlockEntry(); + for (int j = 0; j < toFieldTypes.size(); j++) { + if (j >= fromFieldTypes.size() || singleRowBlock.isNull(j)) { + singleRowBuilder.appendNull(); + } + else if (coercers[j] == null) { + singleRowBlock.writePositionTo(j, singleRowBuilder); + singleRowBuilder.closeEntry(); + } + else { + Block singleFieldBlock = coercers[j].apply(singleRowBlock.getSingleValueBlock(j)); + if (singleFieldBlock.isNull(0)) { + singleRowBuilder.appendNull(); + } + else { + singleFieldBlock.writePositionTo(0, singleRowBuilder); + singleRowBuilder.closeEntry(); + } + } + } + blockBuilder.closeEntry(); + } + return blockBuilder.build(); + } + } + private final class CoercionLazyBlockLoader implements LazyBlockLoader { @@ -450,4 +653,10 @@ public void load(LazyBlock lazyBlock) block = null; } } + + private static List getAllStructFieldTypeInfos(HiveType hiveType) + { + return ((StructTypeInfo) hiveType.getTypeInfo()).getAllStructFieldTypeInfos() + .stream().map(typeInfo -> HiveType.valueOf(typeInfo.getTypeName())).collect(Collectors.toList()); + } } 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 bae8b1238f21..952a00c438d3 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 @@ -118,6 +118,7 @@ import java.util.TimeZone; import java.util.UUID; import java.util.concurrent.ExecutorService; +import java.util.stream.Stream; import static com.facebook.presto.hive.AbstractTestHiveClient.TransactionDeleteInsertTestTag.COMMIT; import static com.facebook.presto.hive.AbstractTestHiveClient.TransactionDeleteInsertTestTag.ROLLBACK_AFTER_APPEND_PAGE; @@ -250,7 +251,7 @@ public abstract class AbstractTestHiveClient .add(new ColumnMetadata("t_row", ROW_TYPE)) .build(); - private static final List MISMATCH_SCHEMA_TABLE_BEFORE = ImmutableList.builder() + private static final List MISMATCH_SCHEMA_PRIMITIVE_COLUMN_BEFORE = ImmutableList.builder() .add(new ColumnMetadata("tinyint_to_smallint", TINYINT)) .add(new ColumnMetadata("tinyint_to_integer", TINYINT)) .add(new ColumnMetadata("tinyint_to_bigint", TINYINT)) @@ -260,10 +261,10 @@ public abstract class AbstractTestHiveClient .add(new ColumnMetadata("integer_to_varchar", INTEGER)) .add(new ColumnMetadata("varchar_to_integer", createUnboundedVarcharType())) .add(new ColumnMetadata("float_to_double", REAL)) - .add(new ColumnMetadata("ds", createUnboundedVarcharType())) + .add(new ColumnMetadata("varchar_to_varchar", createUnboundedVarcharType())) .build(); - private static final List MISMATCH_SCHEMA_TABLE_AFTER = ImmutableList.builder() + private static final List MISMATCH_SCHEMA_PRIMITIVE_COLUMN_AFTER = ImmutableList.builder() .add(new ColumnMetadata("tinyint_to_smallint", SMALLINT)) .add(new ColumnMetadata("tinyint_to_integer", INTEGER)) .add(new ColumnMetadata("tinyint_to_bigint", BIGINT)) @@ -273,6 +274,52 @@ public abstract class AbstractTestHiveClient .add(new ColumnMetadata("integer_to_varchar", createUnboundedVarcharType())) .add(new ColumnMetadata("varchar_to_integer", INTEGER)) .add(new ColumnMetadata("float_to_double", DOUBLE)) + .add(new ColumnMetadata("varchar_to_varchar", createUnboundedVarcharType())) + .build(); + + private static final Type MISMATCH_SCHEMA_ROW_TYPE_BEFORE = TYPE_MANAGER.getParameterizedType( + ROW, + MISMATCH_SCHEMA_PRIMITIVE_COLUMN_BEFORE + .stream() + .map(col -> TypeSignatureParameter.of(new NamedTypeSignature(format("f_%s", col.getName()), col.getType().getTypeSignature()))) + .collect(toList())); + private static final Type MISMATCH_SCHEMA_ROW_TYPE_AFTER = TYPE_MANAGER.getParameterizedType( + ROW, + Stream.concat( + MISMATCH_SCHEMA_PRIMITIVE_COLUMN_AFTER.stream(), + MISMATCH_SCHEMA_PRIMITIVE_COLUMN_AFTER.stream().limit(1).map(columnMetadata -> new ColumnMetadata(format("%s_append", columnMetadata.getName()), columnMetadata.getType()))) + .map(col -> TypeSignatureParameter.of(new NamedTypeSignature(format("f_%s", col.getName()), col.getType().getTypeSignature()))) + .collect(toList())); + private static final Type MISMATCH_SCHEMA_ARRAY_TYPE_BEFORE = TYPE_MANAGER.getParameterizedType( + ARRAY, + ImmutableList.of(TypeSignatureParameter.of(MISMATCH_SCHEMA_ROW_TYPE_BEFORE.getTypeSignature()))); + private static final Type MISMATCH_SCHEMA_ARRAY_TYPE_AFTER = TYPE_MANAGER.getParameterizedType( + ARRAY, + ImmutableList.of(TypeSignatureParameter.of(MISMATCH_SCHEMA_ROW_TYPE_AFTER.getTypeSignature()))); + private static final Type MISMATCH_SCHEMA_MAP_TYPE_BEFORE = TYPE_MANAGER.getParameterizedType( + MAP, + ImmutableList.of( + TypeSignatureParameter.of(MISMATCH_SCHEMA_ROW_TYPE_BEFORE.getTypeParameters().get(1).getTypeSignature()), + TypeSignatureParameter.of(MISMATCH_SCHEMA_ROW_TYPE_BEFORE.getTypeSignature()))); + private static final Type MISMATCH_SCHEMA_MAP_TYPE_AFTER = TYPE_MANAGER.getParameterizedType( + MAP, + ImmutableList.of( + TypeSignatureParameter.of(MISMATCH_SCHEMA_ROW_TYPE_AFTER.getTypeParameters().get(1).getTypeSignature()), + TypeSignatureParameter.of(MISMATCH_SCHEMA_ROW_TYPE_AFTER.getTypeSignature()))); + + private static final List MISMATCH_SCHEMA_TABLE_BEFORE = ImmutableList.builder() + .addAll(MISMATCH_SCHEMA_PRIMITIVE_COLUMN_BEFORE) + .add(new ColumnMetadata("struct_to_struct", MISMATCH_SCHEMA_ROW_TYPE_BEFORE)) + .add(new ColumnMetadata("list_to_list", MISMATCH_SCHEMA_ARRAY_TYPE_BEFORE)) + .add(new ColumnMetadata("map_to_map", MISMATCH_SCHEMA_MAP_TYPE_BEFORE)) + .add(new ColumnMetadata("ds", createUnboundedVarcharType())) + .build(); + + private static final List MISMATCH_SCHEMA_TABLE_AFTER = ImmutableList.builder() + .addAll(MISMATCH_SCHEMA_PRIMITIVE_COLUMN_AFTER) + .add(new ColumnMetadata("struct_to_struct", MISMATCH_SCHEMA_ROW_TYPE_AFTER)) + .add(new ColumnMetadata("list_to_list", MISMATCH_SCHEMA_ARRAY_TYPE_AFTER)) + .add(new ColumnMetadata("map_to_map", MISMATCH_SCHEMA_MAP_TYPE_AFTER)) .add(new ColumnMetadata("ds", createUnboundedVarcharType())) .build(); @@ -283,7 +330,7 @@ public abstract class AbstractTestHiveClient .row(3L, "bye", (byte) 46, (short) 346, 345, 456L, 754.2008f, 98.1, false, ImmutableList.of("ape", "bear"), ImmutableMap.of("three", 3L, "four", 4L), ImmutableList.of("false", 0L, false)) .build(); - private static final MaterializedResult MISMATCH_SCHEMA_TABLE_DATA_BEFORE = + private static final MaterializedResult MISMATCH_SCHEMA_PRIMITIVE_FIELDS_DATA_BEFORE = MaterializedResult.resultBuilder(SESSION, TINYINT, TINYINT, TINYINT, SMALLINT, SMALLINT, INTEGER, INTEGER, createUnboundedVarcharType(), REAL, createUnboundedVarcharType()) .row((byte) -11, (byte) 12, (byte) -13, (short) 14, (short) 15, -16, 17, "2147483647", 18.0f, "2016-08-01") .row((byte) 21, (byte) -22, (byte) 23, (short) -24, (short) 25, 26, -27, "asdf", -28.0f, "2016-08-02") @@ -291,7 +338,7 @@ public abstract class AbstractTestHiveClient .row(null, (byte) 42, (byte) 43, (short) 44, (short) -45, 46, 47, "2147483648", 49.5f, "2016-08-03") .build(); - private static final MaterializedResult MISMATCH_SCHEMA_TABLE_DATA_AFTER = + private static final MaterializedResult MISMATCH_SCHEMA_PRIMITIVE_FIELDS_DATA_AFTER = MaterializedResult.resultBuilder(SESSION, SMALLINT, INTEGER, BIGINT, INTEGER, BIGINT, BIGINT, createUnboundedVarcharType(), INTEGER, DOUBLE, createUnboundedVarcharType()) .row((short) -11, 12, -13L, 14, 15L, -16L, "17", 2147483647, 18.0, "2016-08-01") .row((short) 21, -22, 23L, -24, 25L, 26L, "-27", null, -28.0, "2016-08-02") @@ -299,6 +346,36 @@ public abstract class AbstractTestHiveClient .row(null, 42, 43L, 44, -45L, 46L, "47", null, 49.5, "2016-08-03") .build(); + private static final MaterializedResult MISMATCH_SCHEMA_TABLE_DATA_BEFORE = + MaterializedResult.resultBuilder(SESSION, MISMATCH_SCHEMA_TABLE_BEFORE.stream().map(ColumnMetadata::getType).collect(toList())) + .rows(MISMATCH_SCHEMA_PRIMITIVE_FIELDS_DATA_BEFORE.getMaterializedRows() + .stream() + .map(materializedRow -> { + List result = materializedRow.getFields(); + List rowResult = materializedRow.getFields(); + result.add(rowResult); + result.add(ImmutableList.of(rowResult, rowResult)); + result.add(ImmutableMap.of(rowResult.get(1), rowResult)); + result.add(rowResult.get(9)); + return new MaterializedRow(materializedRow.getPrecision(), result); + }).collect(toList())) + .build(); + private static final MaterializedResult MISMATCH_SCHEMA_TABLE_DATA_AFTER = + MaterializedResult.resultBuilder(SESSION, MISMATCH_SCHEMA_TABLE_AFTER.stream().map(ColumnMetadata::getType).collect(toList())) + .rows(MISMATCH_SCHEMA_PRIMITIVE_FIELDS_DATA_AFTER.getMaterializedRows() + .stream() + .map(materializedRow -> { + List result = materializedRow.getFields(); + List rowResult = materializedRow.getFields(); + rowResult.add(null); + result.add(rowResult); + result.add(ImmutableList.of(rowResult, rowResult)); + result.add(ImmutableMap.of(rowResult.get(1), rowResult)); + result.add(rowResult.get(9)); + return new MaterializedRow(materializedRow.getPrecision(), result); + }).collect(toList())) + .build(); + private static final List CREATE_TABLE_COLUMNS_PARTITIONED = ImmutableList.builder() .addAll(CREATE_TABLE_COLUMNS) .add(new ColumnMetadata("ds", createUnboundedVarcharType())) From 6851b1465ad93cdce00f1abcb8bcd68f9623d83a Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Sat, 14 Oct 2017 12:45:25 -0700 Subject: [PATCH 241/331] Update TestHiveCoercion --- presto-product-tests/pom.xml | 4 + .../presto/tests/hive/TestHiveCoercion.java | 88 +++++++++++++++++-- 2 files changed, 87 insertions(+), 5 deletions(-) diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index fbec4548445e..61a5fac5f44f 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -61,6 +61,10 @@ org.testng testng + + io.airlift + testing + javax.inject javax.inject diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveCoercion.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveCoercion.java index c0cba5e55140..582a9a96f9a8 100644 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveCoercion.java +++ b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveCoercion.java @@ -13,6 +13,9 @@ */ package com.facebook.presto.tests.hive; +import com.facebook.presto.jdbc.PrestoArray; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.teradata.tempto.ProductTest; import com.teradata.tempto.Requirement; import com.teradata.tempto.RequirementsProvider; @@ -31,8 +34,13 @@ import java.sql.Connection; import java.sql.SQLException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; import java.util.Locale; +import java.util.Map; import java.util.Optional; +import java.util.stream.Collectors; import static com.facebook.presto.tests.TestGroups.HIVE_COERCION; import static com.facebook.presto.tests.TestGroups.HIVE_CONNECTOR; @@ -45,18 +53,23 @@ import static com.teradata.tempto.fulfillment.table.TableHandle.tableHandle; import static com.teradata.tempto.query.QueryExecutor.defaultQueryExecutor; import static com.teradata.tempto.query.QueryExecutor.query; +import static io.airlift.testing.Assertions.assertEqualsIgnoreOrder; import static java.lang.String.format; +import static java.sql.JDBCType.ARRAY; import static java.sql.JDBCType.BIGINT; import static java.sql.JDBCType.DOUBLE; import static java.sql.JDBCType.INTEGER; +import static java.sql.JDBCType.JAVA_OBJECT; import static java.sql.JDBCType.LONGNVARCHAR; import static java.sql.JDBCType.SMALLINT; import static java.sql.JDBCType.VARBINARY; +import static org.testng.Assert.assertEquals; public class TestHiveCoercion extends ProductTest { private static String tableNameFormat = "%s_hive_coercion"; + private static String dummyTableNameFormat = "%s_dummy"; public static final HiveTableDefinition HIVE_COERCION_TEXTFILE = tableDefinitionBuilder("TEXTFILE", Optional.empty(), Optional.of("DELIMITED FIELDS TERMINATED BY '|'")) .setNoData() @@ -91,7 +104,10 @@ private static HiveTableDefinition.HiveTableDefinitionBuilder tableDefinitionBui " smallint_to_bigint SMALLINT," + " int_to_bigint INT," + " bigint_to_varchar BIGINT," + - " float_to_double FLOAT" + + " float_to_double FLOAT," + + " row_to_row STRUCT < tinyint_to_smallint : TINYINT, tinyint_to_int : TINYINT, tinyint_to_bigint : TINYINT, smallint_to_int : SMALLINT, smallint_to_bigint : SMALLINT, int_to_bigint : INT, bigint_to_varchar : BIGINT, float_to_double : FLOAT >," + + " list_to_list ARRAY < STRUCT < tinyint_to_smallint : TINYINT, tinyint_to_int : TINYINT, tinyint_to_bigint : TINYINT, smallint_to_int : SMALLINT, smallint_to_bigint : SMALLINT, int_to_bigint : INT, bigint_to_varchar : BIGINT, float_to_double : FLOAT > >," + + " map_to_map MAP < TINYINT, STRUCT < tinyint_to_smallint : TINYINT, tinyint_to_int : TINYINT, tinyint_to_bigint : TINYINT, smallint_to_int : SMALLINT, smallint_to_bigint : SMALLINT, int_to_bigint : INT, bigint_to_varchar : BIGINT, float_to_double : FLOAT > >" + ") " + "PARTITIONED BY (id BIGINT) " + (rowFormat.isPresent() ? "ROW FORMAT " + rowFormat.get() + " " : " ") + @@ -110,12 +126,26 @@ private static HiveTableDefinition.HiveTableDefinitionBuilder parquetTableDefini " smallint_to_bigint SMALLINT," + " int_to_bigint INT," + " bigint_to_varchar BIGINT," + - " float_to_double DOUBLE" + + " float_to_double DOUBLE," + + " row_to_row STRUCT < tinyint_to_smallint : TINYINT, tinyint_to_int : TINYINT, tinyint_to_bigint : TINYINT, smallint_to_int : SMALLINT, smallint_to_bigint : SMALLINT, int_to_bigint : INT, bigint_to_varchar : BIGINT, float_to_double : DOUBLE >," + + " list_to_list ARRAY < STRUCT < tinyint_to_smallint : TINYINT, tinyint_to_int : TINYINT, tinyint_to_bigint : TINYINT, smallint_to_int : SMALLINT, smallint_to_bigint : SMALLINT, int_to_bigint : INT, bigint_to_varchar : BIGINT, float_to_double : DOUBLE > >," + + " map_to_map MAP < TINYINT, STRUCT < tinyint_to_smallint : TINYINT, tinyint_to_int : TINYINT, tinyint_to_bigint : TINYINT, smallint_to_int : SMALLINT, smallint_to_bigint : SMALLINT, int_to_bigint : INT, bigint_to_varchar : BIGINT, float_to_double : DOUBLE > >" + ") " + "PARTITIONED BY (id BIGINT) " + "STORED AS PARQUET"); } + private static String getCreateDummyTableDDL(HiveTableDefinition tableDefinition) + { + String tableName = mutableTableInstanceOf(tableDefinition).getNameInDatabase(); + String floatToDoubleType = tableName.toLowerCase(Locale.ENGLISH).contains("parquet") ? "DOUBLE" : "FLOAT"; + return tableDefinition.getCreateTableDDL(format(dummyTableNameFormat, tableName), Optional.empty()) + .replace(format(" float_to_double %s,", floatToDoubleType), format(" float_to_double %s", floatToDoubleType)) + .replace(format(" row_to_row STRUCT < tinyint_to_smallint : TINYINT, tinyint_to_int : TINYINT, tinyint_to_bigint : TINYINT, smallint_to_int : SMALLINT, smallint_to_bigint : SMALLINT, int_to_bigint : INT, bigint_to_varchar : BIGINT, float_to_double : %s >,", floatToDoubleType), "") + .replace(format(" list_to_list ARRAY < STRUCT < tinyint_to_smallint : TINYINT, tinyint_to_int : TINYINT, tinyint_to_bigint : TINYINT, smallint_to_int : SMALLINT, smallint_to_bigint : SMALLINT, int_to_bigint : INT, bigint_to_varchar : BIGINT, float_to_double : %s > >,", floatToDoubleType), "") + .replace(format(" map_to_map MAP < TINYINT, STRUCT < tinyint_to_smallint : TINYINT, tinyint_to_int : TINYINT, tinyint_to_bigint : TINYINT, smallint_to_int : SMALLINT, smallint_to_bigint : SMALLINT, int_to_bigint : INT, bigint_to_varchar : BIGINT, float_to_double : %s > >", floatToDoubleType), ""); + } + public static final class TextRequirements implements RequirementsProvider { @@ -210,20 +240,37 @@ private void doTestHiveCoercion(HiveTableDefinition tableDefinition) throws SQLException { String tableName = mutableTableInstanceOf(tableDefinition).getNameInDatabase(); - + String dummyTableName = format(dummyTableNameFormat, tableName); + executeHiveQuery(format("DROP TABLE IF EXISTS %s", dummyTableName)); + executeHiveQuery(getCreateDummyTableDDL(tableDefinition)); executeHiveQuery(format("INSERT INTO TABLE %s " + "PARTITION (id=1) " + "VALUES" + "(-1, 2, -3, 100, -101, 2323, 12345, 0.5)," + "(1, -2, null, -100, 101, -2323, -12345, -1.5)", - tableName)); + dummyTableName)); + executeHiveQuery(format("INSERT INTO TABLE %s " + + "PARTITION (id=1) " + + "SELECT" + + " tinyint_to_smallint," + + " tinyint_to_int," + + " tinyint_to_bigint," + + " smallint_to_int," + + " smallint_to_bigint," + + " int_to_bigint," + + " bigint_to_varchar," + + " float_to_double," + + " named_struct('tinyint_to_smallint', tinyint_to_smallint, 'tinyint_to_int', tinyint_to_int, 'tinyint_to_bigint', tinyint_to_bigint, 'smallint_to_int', smallint_to_int, 'smallint_to_bigint', smallint_to_bigint, 'int_to_bigint', int_to_bigint, 'bigint_to_varchar', bigint_to_varchar, 'float_to_double', float_to_double)," + + " array(named_struct('tinyint_to_smallint', tinyint_to_smallint, 'tinyint_to_int', tinyint_to_int, 'tinyint_to_bigint', tinyint_to_bigint, 'smallint_to_int', smallint_to_int, 'smallint_to_bigint', smallint_to_bigint, 'int_to_bigint', int_to_bigint, 'bigint_to_varchar', bigint_to_varchar, 'float_to_double', float_to_double))," + + " map(tinyint_to_int, named_struct('tinyint_to_smallint', tinyint_to_smallint, 'tinyint_to_int', tinyint_to_int, 'tinyint_to_bigint', tinyint_to_bigint, 'smallint_to_int', smallint_to_int, 'smallint_to_bigint', smallint_to_bigint, 'int_to_bigint', int_to_bigint, 'bigint_to_varchar', bigint_to_varchar, 'float_to_double', float_to_double))" + + "FROM %s", tableName, dummyTableName)); alterTableColumnTypes(tableName); assertProperAlteredTableSchema(tableName); QueryResult queryResult = query(format("SELECT * FROM %s", tableName)); assertColumnTypes(queryResult); - assertThat(queryResult).containsOnly( + assertThat(queryResult.project(1, 2, 3, 4, 5, 6, 7, 8, 12)).containsOnly( row( -1, 2, @@ -244,6 +291,15 @@ private void doTestHiveCoercion(HiveTableDefinition tableDefinition) "-12345", -1.5, 1)); + List rowColumn = ImmutableList.of( + namedStruct("tinyint_to_smallint", (short) -1, "tinyint_to_int", 2, "tinyint_to_bigint", -3L, "smallint_to_int", 100, "smallint_to_bigint", -101L, "int_to_bigint", 2323L, "bigint_to_varchar", "12345", "float_to_double", 0.5), + namedStruct("tinyint_to_smallint", (short) 1, "tinyint_to_int", -2, "tinyint_to_bigint", null, "smallint_to_int", -100, "smallint_to_bigint", 101L, "int_to_bigint", -2323L, "bigint_to_varchar", "-12345", "float_to_double", -1.5)); + assertEqualsIgnoreOrder(queryResult.column(9), rowColumn, "row_to_row field is not equal"); + assertEqualsIgnoreOrder( + queryResult.column(10).stream().map(o -> Arrays.asList((Object[]) ((PrestoArray) o).getArray())).collect(Collectors.toList()), + rowColumn.stream().map(ImmutableList::of).collect(Collectors.toList()), + "list_to_list field is not equal"); + assertEqualsIgnoreOrder(queryResult.column(11), rowColumn.stream().map(map -> ImmutableMap.of(map.get("tinyint_to_int"), map)).collect(Collectors.toList()), "map_to_map field is not equal"); } private void assertProperAlteredTableSchema(String tableName) @@ -257,6 +313,9 @@ private void assertProperAlteredTableSchema(String tableName) row("int_to_bigint", "bigint"), row("bigint_to_varchar", "varchar"), row("float_to_double", "double"), + row("row_to_row", "row(tinyint_to_smallint smallint, tinyint_to_int integer, tinyint_to_bigint bigint, smallint_to_int integer, smallint_to_bigint bigint, int_to_bigint bigint, bigint_to_varchar varchar, float_to_double double)"), + row("list_to_list", "array(row(tinyint_to_smallint smallint, tinyint_to_int integer, tinyint_to_bigint bigint, smallint_to_int integer, smallint_to_bigint bigint, int_to_bigint bigint, bigint_to_varchar varchar, float_to_double double))"), + row("map_to_map", "map(integer, row(tinyint_to_smallint smallint, tinyint_to_int integer, tinyint_to_bigint bigint, smallint_to_int integer, smallint_to_bigint bigint, int_to_bigint bigint, bigint_to_varchar varchar, float_to_double double))"), row("id", "bigint")); } @@ -273,6 +332,9 @@ private void assertColumnTypes(QueryResult queryResult) BIGINT, LONGNVARCHAR, DOUBLE, + JAVA_OBJECT, + ARRAY, + JAVA_OBJECT, BIGINT); } else if (usingTeradataJdbcDriver(connection)) { @@ -285,6 +347,9 @@ else if (usingTeradataJdbcDriver(connection)) { BIGINT, VARBINARY, DOUBLE, + JAVA_OBJECT, + ARRAY, + JAVA_OBJECT, BIGINT); } else { @@ -302,6 +367,9 @@ private static void alterTableColumnTypes(String tableName) executeHiveQuery(format("ALTER TABLE %s CHANGE COLUMN int_to_bigint int_to_bigint bigint", tableName)); executeHiveQuery(format("ALTER TABLE %s CHANGE COLUMN bigint_to_varchar bigint_to_varchar string", tableName)); executeHiveQuery(format("ALTER TABLE %s CHANGE COLUMN float_to_double float_to_double double", tableName)); + executeHiveQuery(format("ALTER TABLE %s CHANGE COLUMN row_to_row row_to_row struct", tableName)); + executeHiveQuery(format("ALTER TABLE %s CHANGE COLUMN list_to_list list_to_list array>", tableName)); + executeHiveQuery(format("ALTER TABLE %s CHANGE COLUMN map_to_map map_to_map map>", tableName)); } private static TableInstance mutableTableInstanceOf(TableDefinition tableDefinition) @@ -337,4 +405,14 @@ private static QueryResult executeHiveQuery(String query) { return testContext().getDependency(QueryExecutor.class, "hive").executeQuery(query); } + + private static Map namedStruct(Object... objects) + { + assertEquals(objects.length % 2, 0, "number of objects must be even"); + Map struct = new HashMap<>(); + for (int i = 0; i < objects.length; i += 2) { + struct.put(objects[i], objects[i + 1]); + } + return struct; + } } From ef2caad2df417bfe2ded062cf547bbdfcd39b997 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 20 Oct 2017 17:13:32 -0700 Subject: [PATCH 242/331] Remove CLI group in singlenode-hdfs-impersonation environment product test --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index ecaef70c6f2f..7ec6c74c23a7 100644 --- a/.travis.yml +++ b/.travis.yml @@ -100,7 +100,7 @@ script: - | if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then presto-product-tests/bin/run_on_docker.sh \ - singlenode-hdfs-impersonation -g storage_formats,cli,hdfs_impersonation + singlenode-hdfs-impersonation -g storage_formats,hdfs_impersonation fi - | if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then From 52b1266e0d8779a52cb0625b587a285dfaccda17 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 24 Oct 2017 14:47:30 -0700 Subject: [PATCH 243/331] Move back to 0.186 version number to prepare upgrade --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 55 files changed, 57 insertions(+), 57 deletions(-) diff --git a/pom.xml b/pom.xml index ef91ee22d39a..81dd7436a22b 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.186-tw-0.40 + 0.186 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 5e42dee28db8..4e3f10105fbf 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 011076004286..3467144cdf78 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 54a51004bf1c..c0c0229e8235 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 599b90776974..eb2ae8f710df 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index f61eac6950e6..ee9c1dc5c338 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index dfad3a307b59..f02cdb406b7f 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.186-tw-0.40 + 0.186 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 4518855f718a..efc7cd49b351 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index ebdd94c220c3..b0632e764c1b 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index e65e6c921e7b..95051e7de3bb 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 7f3316877167..eab18e0e8713 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index d5e81432ade5..8d6952e9000e 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index d95b3829d95e..e7a3c7907413 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 29d50e323a91..135c90900d6e 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index d2eababf0733..1c3d58e4d66c 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-example-http diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index e8d6c8a4e1cd..0356cf9bce69 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 1879014e9600..23772a96aef8 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index b24dbcbc524c..07f44a61cf95 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index c98b6edb7c8b..bb5d61d20666 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 49c01b85454c..a6c34ef6b3cd 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index e0c4b94faa78..d354a8e05d7b 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index 20d6836ce41a..13db7f2624e7 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index f0281298b9a6..e724bc0f8b8c 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 39db44d781a3..0519e4ff8528 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index 9810d2cc6522..5dcadc6658cc 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.186-tw-0.40 + 0.186 presto-matching diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 18c96fb391bb..d79357a77d6f 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 510a66e8883c..2664dad8453a 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index e0039e9c7474..38663898ec8d 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index ad77789e5f35..5a778ad18797 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 35bc32ab5f61..274e49e5c628 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index f4b9dfc8bed5..915f4bf53447 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 26828c1b7ea9..36d6463a6d67 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index b3166c93019b..f5d66dfe9504 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index fbec4548445e..a8fb1b83161e 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.186-tw-0.40 + 0.186 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index bc5f3996d0f9..0b2e8fc99995 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index a2144c466cf9..dfa72802b72c 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 67994203e1e3..f64076dd5cb1 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 10f40b16c1fb..4f67bce04495 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index 49a56619df60..666192caac6c 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index a1389a95f0df..96fe4ac23b5f 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 700b1f45ce49..fcc36f9cdc47 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 53fa54923ea8..22e522539805 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 09de8fc4fc43..8faca4c35e5c 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 19673a3844fa..43f1cea09dec 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.186-tw-0.40 + 0.186 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 41614845a60e..30e5c6b4468e 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 57beeba35d18..40d7337dd76b 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 81af859ca8e7..f7a7c013a432 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.186-tw-0.40 + 0.186 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index 7d3964550c86..ed6173555651 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 0d0e5214948f..8b139fcecdac 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index fbd6d458329c..71527cd70eeb 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index 5935f44da828..0bdcc0773e1b 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index aa3fa56a5f0a..b62bc93cf3cd 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-tpch diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index ae79faf0f5af..54e05fd44bf9 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index c6d24b4e4340..1de03c404aa7 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 34aa8ee06458..194bedc6f336 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.186-tw-0.40 + 0.186 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.186-tw-0.40 + 0.186 provided From d2083d3b03a138b3bccbb6a54d3cc18c39d6675e Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 24 Oct 2017 15:28:45 -0700 Subject: [PATCH 244/331] bump up version to 0.187-tw-0.41 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 55 files changed, 57 insertions(+), 57 deletions(-) diff --git a/pom.xml b/pom.xml index 6f47dcc183df..db06e85bdba7 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.187 + 0.187-tw-0.41 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 2f2c0ad5b65e..0b8ab68c0553 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 0895ff78e0d3..c504874ac4a8 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index c3736ed0b498..b62e8165a71d 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index fa135d9905e0..89253245f78c 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 57dc61b6d1bc..5bea8b48fdca 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index f0cbb1a31e5b..faec4cb2ed54 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.187 + 0.187-tw-0.41 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 68825acac829..773ec05f4164 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 32405b841d48..99f4b9b10601 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 1f75449ea02f..48dbd3bfe437 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 859a397becee..a901babd660b 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 8bc4461bc7a6..dc1574a348bc 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index d3e9debbba56..3ab19efa9591 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 77d75a0b5084..9f2aa31181ff 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index ec626fc9bce1..da6d5803fe27 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-example-http diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index 314dc85685f9..4cf42a01bd13 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index e9d6501964c6..31fa7413549d 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 99f89c6082f9..c3e90ad08f15 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 8b7c806532b1..6f47acddb5d5 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 0292d930601f..120557653f11 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 434a0edd4464..3296c6d6e23c 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index 13db7f2624e7..e9ea231fad10 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.187-tw-0.41 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 095d94b5526d..cd82eb9bf9a9 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 986548d69134..21d4b2a6c540 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index 0ba9bd640b3c..36ed56c800b3 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.187 + 0.187-tw-0.41 presto-matching diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 717a7a0d05ce..911576371770 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 7af1cedb6c50..f5701cd18cb5 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 7e3ea5b3f067..bc929fbcfde8 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 5f0cb15ce6fe..5f5012e3474d 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index ea3d33b6b079..c3671a067735 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 98afecfeec0b..aeba6f941f32 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index e57faadc4b66..d7025ffd0850 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 0ac4cb24b971..7d49988b59d8 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index af1ade90e4d6..bc023eeff62a 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.187 + 0.187-tw-0.41 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 989c2718b1c3..a70d2311a126 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index d93e70d6a898..8a83e35edddf 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index cb58ea454217..7f9d5ab69636 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 270aeeb6539a..e2d3a835c3db 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index 2cabd896e1e1..e5162a902a08 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 698342827ed5..d1335e189d86 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 4c02c87be26f..86d5790bfb8b 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 25b961d3ef75..ae6835d55c9c 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 82e989a02963..ae6490d2e8d2 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 08534be2d256..8c3f066325ab 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.187 + 0.187-tw-0.41 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index a66309b2967c..a245073da4a7 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 70c53762ee99..97c5b45680b3 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index e31f27e8fc67..9bc9d201a69d 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.187 + 0.187-tw-0.41 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index 971831f9f68d..aca3ebe9dc13 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 7809b2245aaa..473cc02da789 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index cd6283e4e9b4..b84c0e3ac2f4 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index 2aeda55c8bdd..16090a9941b6 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 57b8d1c2edca..ad7027c80e57 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-tpch diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index 54e05fd44bf9..50d34ccdd628 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.186 + 0.187-tw-0.41 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 6f35edee3e74..dc23d4c812e8 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187 + 0.187-tw-0.41 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 194bedc6f336..b3aa09b9f7bc 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.186 + 0.187-tw-0.41 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.186 + 0.187-tw-0.41 provided From 5232a9f2a7bf73e128a6d2f0ca00ec5c357a0eb8 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 31 Oct 2017 14:20:48 -0700 Subject: [PATCH 245/331] Fix block builder --- .../hive/thrift/ThriftHiveRecordCursor.java | 41 +++++++++---------- 1 file changed, 19 insertions(+), 22 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java index d48be26ebc2d..ae18e8dc272b 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java @@ -603,13 +603,12 @@ private static Block serializeMap(Type type, ThriftFieldIdResolver resolver, Blo Type valueType = typeParameters.get(1); ThriftFieldIdResolver keyResolver = resolver.getNestedResolver(0); ThriftFieldIdResolver valueResolver = resolver.getNestedResolver(1); - BlockBuilder currentBuilder; - if (builder != null) { - currentBuilder = builder.beginBlockEntry(); - } - else { - currentBuilder = type.createBlockBuilder(new BlockBuilderStatus(), map.size()); + boolean builderSynthesized = false; + if (builder == null) { + builderSynthesized = true; + builder = type.createBlockBuilder(new BlockBuilderStatus(), 1); } + BlockBuilder currentBuilder = builder.beginBlockEntry(); for (Map.Entry entry : map.entrySet()) { // Hive skips map entries with null keys @@ -619,13 +618,12 @@ private static Block serializeMap(Type type, ThriftFieldIdResolver resolver, Blo } } - if (builder != null) { - builder.closeEntry(); - return null; + builder.closeEntry(); + if (builderSynthesized) { + return (Block) type.getObject(builder, 0); } else { - Block resultBlock = currentBuilder.build(); - return resultBlock; + return null; } } @@ -633,13 +631,13 @@ private static Block serializeStruct(Type type, ThriftFieldIdResolver resolver, { ThriftGenericRow structData = (ThriftGenericRow) requireNonNull(object, "object is null"); List typeParameters = type.getTypeParameters(); - BlockBuilder currentBuilder; - if (builder != null) { - currentBuilder = builder.beginBlockEntry(); - } - else { - currentBuilder = type.createBlockBuilder(new BlockBuilderStatus(), typeParameters.size()); + + boolean builderSynthesized = false; + if (builder == null) { + builderSynthesized = true; + builder = type.createBlockBuilder(new BlockBuilderStatus(), 1); } + BlockBuilder currentBuilder = builder.beginBlockEntry(); for (int i = 0; i < typeParameters.size(); i++) { Object fieldValue = structData.getFieldValueForThriftId(resolver.getThriftId(i)); @@ -651,13 +649,12 @@ private static Block serializeStruct(Type type, ThriftFieldIdResolver resolver, } } - if (builder != null) { - builder.closeEntry(); - return null; + builder.closeEntry(); + if (builderSynthesized) { + return (Block) type.getObject(builder, 0); } else { - Block resultBlock = currentBuilder.build(); - return resultBlock; + return null; } } From 1a1474af9b7466394515479149d413a0804765f8 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 31 Oct 2017 15:01:53 -0700 Subject: [PATCH 246/331] add geospatial plugin in presto-twitter-server --- presto-twitter-server/src/main/provisio/presto.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/presto-twitter-server/src/main/provisio/presto.xml b/presto-twitter-server/src/main/provisio/presto.xml index 64124eefd08b..85dc97c16451 100644 --- a/presto-twitter-server/src/main/provisio/presto.xml +++ b/presto-twitter-server/src/main/provisio/presto.xml @@ -50,14 +50,14 @@ - - + + - - + + From d2a9f198d3c48c15134c76086fbe3e83780e57da Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 1 Nov 2017 13:30:37 -0700 Subject: [PATCH 247/331] bump up version to 0.187-tw-0.42 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 55 files changed, 57 insertions(+), 57 deletions(-) diff --git a/pom.xml b/pom.xml index db06e85bdba7..64723c9bec24 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.187-tw-0.41 + 0.187-tw-0.42 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 0b8ab68c0553..736a322792ac 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index c504874ac4a8..33772ebc1736 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index b62e8165a71d..97ea1e5f6c01 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 89253245f78c..baf22bb80a55 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 5bea8b48fdca..e8b1ff174096 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index faec4cb2ed54..814c2890b3e5 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.187-tw-0.41 + 0.187-tw-0.42 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 773ec05f4164..82c0e3a4ec19 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 99f4b9b10601..f26b884f0bc4 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 48dbd3bfe437..d25d3c87c40f 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index a901babd660b..d0caecf70300 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index dc1574a348bc..34d4e62940d8 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 3ab19efa9591..5424187b3542 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 9f2aa31181ff..7ea9ca138b1b 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index da6d5803fe27..5f7f75f3c211 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-example-http diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index 4cf42a01bd13..b35b45a40f48 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 31fa7413549d..ebdce520e49d 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index c3e90ad08f15..2a883b48e73c 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 6f47acddb5d5..969cdbbec9f3 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 120557653f11..f4fc87a7458b 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 3296c6d6e23c..d6628d07177b 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index e9ea231fad10..3a62de7d3562 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index cd82eb9bf9a9..155d10161d08 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 21d4b2a6c540..3ebd5cdcd69d 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index 36ed56c800b3..d8f177ad5d42 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.187-tw-0.41 + 0.187-tw-0.42 presto-matching diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 911576371770..bb34fa920cfc 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index f5701cd18cb5..7ae84580ca57 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index bc929fbcfde8..88f527aaa2b6 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 5f5012e3474d..3f2c3a05aafc 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index c3671a067735..fe0b692375cc 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index aeba6f941f32..c343f59645d3 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index d7025ffd0850..527bceb4a280 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 7d49988b59d8..9c05ab9d7052 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index bc023eeff62a..1febb3a3e893 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.187-tw-0.41 + 0.187-tw-0.42 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index a70d2311a126..5f5b228b5e4f 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 8a83e35edddf..f345288b8660 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 7f9d5ab69636..079d09aff76d 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index e2d3a835c3db..cba676f871ab 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index e5162a902a08..4eb97366751a 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index d1335e189d86..f2910d1fbbac 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 86d5790bfb8b..068357a15d7f 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index ae6835d55c9c..3b0b530aa6f6 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index ae6490d2e8d2..9f4a6da2cf4a 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 8c3f066325ab..ca02f6f1619c 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.187-tw-0.41 + 0.187-tw-0.42 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index a245073da4a7..e9b27ce8fef0 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 97c5b45680b3..1d94c3a8e0f6 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 9bc9d201a69d..233c2fa4e21c 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.187-tw-0.41 + 0.187-tw-0.42 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index aca3ebe9dc13..a9f19ed69bc3 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 473cc02da789..7b354744f4da 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index b84c0e3ac2f4..9496e2ec036d 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index 16090a9941b6..0c17fe0056c3 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index ad7027c80e57..93db4acaa631 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-tpch diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index 50d34ccdd628..0bb76f7379de 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index dc23d4c812e8..b4089bf36afe 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index b3aa09b9f7bc..295c1e57159a 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-tw-0.41 + 0.187-tw-0.42 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.187-tw-0.41 + 0.187-tw-0.42 provided From 7840b47e0c7cd6452f0d2752999524e9a6ed2c18 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 3 Nov 2017 15:36:46 -0700 Subject: [PATCH 248/331] Unwrap the FileSystemWrapper to get the raw file system when checking isHDFSCompatibleViewFileSystem --- .../src/main/java/com/facebook/presto/hive/HiveWriteUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java index a63400a5fdeb..f9d4ac8893f0 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveWriteUtils.java @@ -448,7 +448,7 @@ public static boolean isViewFileSystem(HdfsContext context, HdfsEnvironment hdfs public static boolean isHDFSCompatibleViewFileSystem(HdfsContext context, HdfsEnvironment hdfsEnvironment, Path path) { try { - return hdfsEnvironment.getFileSystem(context, path) + return getRawFileSystem(hdfsEnvironment.getFileSystem(context, path)) .getClass().getName().equals("org.apache.hadoop.fs.viewfs.HDFSCompatibleViewFileSystem"); } catch (IOException e) { From c1ed0b82024b759b98fc7d4eb72de68285b3bfa4 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 7 Nov 2017 15:47:23 -0800 Subject: [PATCH 249/331] Switch the version number to oss base version 0.188 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 55 files changed, 57 insertions(+), 57 deletions(-) diff --git a/pom.xml b/pom.xml index 64723c9bec24..c6772a20e76d 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.187-tw-0.42 + 0.188 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 736a322792ac..7aff1063715f 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 33772ebc1736..f82338b19199 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 97ea1e5f6c01..b2dd0f44c1c2 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index baf22bb80a55..18daf2cd97aa 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index e8b1ff174096..8e74cc381e76 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 814c2890b3e5..e17ba6de0e2c 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.187-tw-0.42 + 0.188 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 82c0e3a4ec19..0051682b4800 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index f26b884f0bc4..2550b56517e1 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index d25d3c87c40f..b8fc8e151823 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index d0caecf70300..8ade3e8b43be 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 34d4e62940d8..48d22e70392e 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 5424187b3542..ca418f9c2531 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 7ea9ca138b1b..3d360add49d8 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 5f7f75f3c211..234d3117d8a1 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-example-http diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index b35b45a40f48..e7dc3d2a7db2 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index ebdce520e49d..3c8520857f14 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 2a883b48e73c..e5d31fbba233 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 969cdbbec9f3..5566d168f774 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index f4fc87a7458b..1a2d5ffdc212 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index d6628d07177b..98a15af64695 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index 3a62de7d3562..2d8bbecbf795 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 155d10161d08..9a85dc990fd4 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 3ebd5cdcd69d..9e90907b4a76 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index d8f177ad5d42..5e21c2e54dc0 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.187-tw-0.42 + 0.188 presto-matching diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index bb34fa920cfc..7979b692e009 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 7ae84580ca57..f153bf5d222e 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 88f527aaa2b6..ccddfd49df20 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 3f2c3a05aafc..f9cc2f866afe 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index fe0b692375cc..e4dfb550b22e 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index c343f59645d3..7b86eb659f5a 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 527bceb4a280..009ccf666fdc 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 9c05ab9d7052..500d02b5aea7 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 10cc2fb79ee5..cb3c50ebaa94 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.187-tw-0.42 + 0.188 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 5f5b228b5e4f..1d8ce69f8922 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index f345288b8660..edae416523ca 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 079d09aff76d..4641cff0d36a 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index cba676f871ab..b9ccf83748f3 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index 4eb97366751a..5d08e1e45e6d 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index f2910d1fbbac..6292def7b70a 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 068357a15d7f..f932e9e1873f 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 3b0b530aa6f6..ed48c674bcf6 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 9f4a6da2cf4a..d859afadda81 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index ca02f6f1619c..49ebc9cd5fbe 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.187-tw-0.42 + 0.188 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index e9b27ce8fef0..1a7e56e4c434 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 1d94c3a8e0f6..008396d7a3e9 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 233c2fa4e21c..8f2faf81ef38 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.187-tw-0.42 + 0.188 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index a9f19ed69bc3..24bdc1a6de21 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 7b354744f4da..283c7dd1261d 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index 9496e2ec036d..81ff79ce2519 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index 0c17fe0056c3..a2b8488aa31b 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 93db4acaa631..79beb71cd085 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-tpch diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index 0bb76f7379de..ef51bf94713e 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index b4089bf36afe..c04927764c7e 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 295c1e57159a..dfd6981652f4 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.187-tw-0.42 + 0.188 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.187-tw-0.42 + 0.188 provided From 0360705a519f1ded0b8293eaafc47e4731d48137 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 7 Nov 2017 15:59:12 -0800 Subject: [PATCH 250/331] Bump up version to 0.188-tw-0.43 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 55 files changed, 57 insertions(+), 57 deletions(-) diff --git a/pom.xml b/pom.xml index 5ea92f9abe2d..bbc8d4251808 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.188 + 0.188-tw-0.43 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 7aff1063715f..bd39c22d518c 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index f82338b19199..b8e220e16f9b 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index b2dd0f44c1c2..092fdbdf6e7b 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 18daf2cd97aa..d8b8013517e1 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 8e74cc381e76..283f7bf2068b 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index e17ba6de0e2c..0ac504eb6fcd 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.188 + 0.188-tw-0.43 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 0051682b4800..bdf040f33943 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 2550b56517e1..667ef78ec3e8 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index b8fc8e151823..3b315bc3ab04 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 8ade3e8b43be..6677efaf5215 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 48d22e70392e..a4463ddbf9cc 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index ca418f9c2531..a1668221505e 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 3d360add49d8..159a3a95d1f6 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 234d3117d8a1..1cacfb08cb11 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-example-http diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index e7dc3d2a7db2..04519e7bc63d 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 3c8520857f14..5a12438c8778 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index e5d31fbba233..7d818c0941c8 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 72d6a4e026ec..8a80f8a97a2a 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 1a2d5ffdc212..53c39475b2cb 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 98a15af64695..033354e90040 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index 2d8bbecbf795..d70f592f96e7 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 9a85dc990fd4..733eb044b4f4 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 9e90907b4a76..1b548a268fb2 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index 5e21c2e54dc0..fe9d970ad613 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.188 + 0.188-tw-0.43 presto-matching diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 7979b692e009..b92b0b07e1fe 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index f153bf5d222e..9e65869ed85b 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index ccddfd49df20..58b0d7a47cc5 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index f9cc2f866afe..cccade6be601 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index e4dfb550b22e..54fef6e36928 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 7b86eb659f5a..31a7e2d2bd1f 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 009ccf666fdc..15410679ff07 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 500d02b5aea7..3cac06256774 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index cb3c50ebaa94..e443c66519fb 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.188 + 0.188-tw-0.43 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 1d8ce69f8922..48bcae5e99ec 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index edae416523ca..cc05a89ec95d 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 4641cff0d36a..059f438bed67 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index b9ccf83748f3..add057f40286 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index 5d08e1e45e6d..155bbaa1800c 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 6efad51a6816..cfca4fe4dc40 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index f932e9e1873f..64dd20cb6323 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index ed48c674bcf6..82199c74ac35 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index d859afadda81..9419b8408093 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 49ebc9cd5fbe..f37b3ccfc3ea 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.188 + 0.188-tw-0.43 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 1a7e56e4c434..696da318f34e 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 008396d7a3e9..061dfdab9e5f 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 8f2faf81ef38..4b86d165efab 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.188 + 0.188-tw-0.43 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index 24bdc1a6de21..3eb1c188ad72 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 283c7dd1261d..cec04430c5aa 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index 81ff79ce2519..24fbac9d7fd1 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index a2b8488aa31b..01548b1c0d8c 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 79beb71cd085..785f0cc284ba 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-tpch diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index ef51bf94713e..750183ae9cf7 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index c04927764c7e..5bba64c85f93 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index dfd6981652f4..473b1d6034a4 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.188 + 0.188-tw-0.43 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.188 + 0.188-tw-0.43 provided From 059f41bdac378292b3c3ad219786ed8af1de357c Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 21 Nov 2017 19:44:05 -0800 Subject: [PATCH 251/331] Point all missed thrift id to NON_EXISTED_THRIFT_ID --- .../hive/thrift/ThriftHiveRecordCursor.java | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java index ae18e8dc272b..ade6d392b597 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java @@ -165,12 +165,7 @@ public ThriftHiveRecordCursor( types[i] = typeManager.getType(column.getTypeSignature()); hiveTypes[i] = column.getHiveType(); hiveIndexs[i] = column.getHiveColumnIndex(); - try { - thriftIds[i] = thriftFieldIdResolver.getThriftId(hiveIndexs[i]); - } - catch (PrestoException e) { - thriftIds[i] = NON_EXISTED_THRIFT_ID; - } + thriftIds[i] = getThriftIdWithFailOver(thriftFieldIdResolver, hiveIndexs[i]); } } @@ -640,7 +635,7 @@ private static Block serializeStruct(Type type, ThriftFieldIdResolver resolver, BlockBuilder currentBuilder = builder.beginBlockEntry(); for (int i = 0; i < typeParameters.size(); i++) { - Object fieldValue = structData.getFieldValueForThriftId(resolver.getThriftId(i)); + Object fieldValue = structData.getFieldValueForThriftId(getThriftIdWithFailOver(resolver, i)); if (fieldValue == null) { currentBuilder.appendNull(); } @@ -708,4 +703,14 @@ else if (type instanceof DecimalType) { throw new UnsupportedOperationException("Unsupported primitive type: " + type); } } + + private static short getThriftIdWithFailOver(ThriftFieldIdResolver thriftFieldIdResolver, int hiveIndex) + { + try { + return thriftFieldIdResolver.getThriftId(hiveIndex); + } + catch (PrestoException e) { + return NON_EXISTED_THRIFT_ID; + } + } } From 900df86cb38a3ca98ee16195d558a59c14b1fc27 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Mon, 11 Dec 2017 12:56:04 -0800 Subject: [PATCH 252/331] Add slack room notification --- .travis.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.travis.yml b/.travis.yml index 11e283840ca4..3b384a56b3ec 100644 --- a/.travis.yml +++ b/.travis.yml @@ -138,6 +138,8 @@ notifications: hipchat: rooms: secure: peNh1KxwlxIpFyb60S8AMvaJThgh1LsjE+Whf1rYkJalVd2wUrqBIoyDKVSueyHD01hQ06gT7rBV6Pu/QcBMR1a9BbMCjERfxLZFUAheuC2Rsb+p1c4dyvBcFUGacgW7XWKCaVYGDGxuUvb0I3Z8cR6KxhK2xi88tHiqBGVGV2yI6zzOTpWVknMfFBtn+ONU1Ob2P6trclXaDyFd4MxubULri6CQdl35eQAq/VnmR3SZOgyVu3V30MGKwI3zhSli+3VqmW0JmaDGoHN6gznM1+VqABLgmIq0P+n+r5gdZWRCorq10NZCFMhVQ8U6rQHcL7sAniYJJsC/yRt6+pjyzIF4N+LSzZ7T+FLxQqT7k/1ukNgrujLDfTpn76Mo9eYTZmfAdzbm1QKJDACwr8Slqhq1jGzcrFMHunvXhVqjOs24R+JAHblY0O9PXvv7aR29GOQWDCvD7nV5QBUr8Xz5q7ozbLqHTI+yH02Jj4EaZ+azWYdRmnr9wDBxWMYBEgOdj4pII9b298XEDB72TxA3KpLTpdLxBTR+gIk/LjJqb/wb84xUv8gPXkaXccltGd5YI90c84cX8isbzNkAylzyfF2Eyueh0XbnMHfpFqBS7qaVM0/D+UxZkU0WNJ0x7G9XJvkiq49bZz2q1KLE4XuvVnTZSSjVSUAS8RtHfwUV33c= + slack: + secure: R0kQ9zwPGx2wKCiiB8crxmniDdYuWMAzkbYKbMMICmIksI+H/15JLXkx8z9DrE0EyjivqhXSGl+CThKmlJKkwEZCpGxVPd2N+kEmYWr/wfjJzfZ/ZqfS6t4+WyCUIIdNiOJM63cjK+0wEH0HPYa//LIFC+DPcvowOBVHTzoUVySXlo83E5mW/1ZvYd4BsVLuI3u01ZkonPVIdtmEErykexPqKExBvpna7homryJTZKF3lUUKsH8GYaDZYF0QdF1F0srxqgAj+fiU16CsXoN2drmDlzK++fB0ilHUW7fhmLju/zfse1k6m7w5/2JBFp0m0WsPo3qOEJBCjyujIwqisQVU90IrRKofMMdOUrElDyruBNU49OatTjQN97YnOLCkNY3fmSPepZb+xhBtn5Flhk2cT8i127rMWaWdpGS/T9IPY2c+/zyXjZDF1F2JTQ1ofKBLQ3jNnXmk0QLVcxj+wFRFSCABZlvsUGOIyayZHfsVojYvvhxG7wpYCU+dc5HtNxCaoloPcPtMD02fgKbdogeRUT/Rkbf8ta74BzoH0XcQx8zSQFN+Dz8xEowBHQq+juWVcH2IIjE08x0Aob4MMyAXuFxagR/WfTDKDpcHSE1SME4D8304fWj91M66ll4Sz89sRw2iIBHGG4dB/WWeE4r3gF4ciRFRYJCr/rTeCJY= before_deploy: - mkdir /tmp/artifacts From 5f8c495485d8ac3ceb64987fc7ff5dca0f1d75c8 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 13 Dec 2017 15:46:13 -0800 Subject: [PATCH 253/331] Handle empty structs by inserting an unknown field --- .../com/facebook/presto/type/RowParametricType.java | 4 +++- .../com/facebook/presto/spi/type/TypeSignature.java | 1 + .../facebook/presto/spi/type/TestTypeSignature.java | 11 ++++++++--- 3 files changed, 12 insertions(+), 4 deletions(-) diff --git a/presto-main/src/main/java/com/facebook/presto/type/RowParametricType.java b/presto-main/src/main/java/com/facebook/presto/type/RowParametricType.java index 173983ee6fde..46e5c02ff8dc 100644 --- a/presto-main/src/main/java/com/facebook/presto/type/RowParametricType.java +++ b/presto-main/src/main/java/com/facebook/presto/type/RowParametricType.java @@ -46,7 +46,9 @@ public String getName() @Override public Type createType(TypeManager typeManager, List parameters) { - checkArgument(!parameters.isEmpty(), "Row type must have at least one parameter"); + if (parameters.isEmpty()) { + parameters.add(TypeParameter.of(new NamedType(UnknownType.NAME, UnknownType.UNKNOWN))); + } checkArgument( parameters.stream().allMatch(parameter -> parameter.getKind() == ParameterKind.NAMED_TYPE), "Expected only named types as a parameters, got %s", diff --git a/presto-spi/src/main/java/com/facebook/presto/spi/type/TypeSignature.java b/presto-spi/src/main/java/com/facebook/presto/spi/type/TypeSignature.java index 1570d045bbb0..74a613ff4b1c 100644 --- a/presto-spi/src/main/java/com/facebook/presto/spi/type/TypeSignature.java +++ b/presto-spi/src/main/java/com/facebook/presto/spi/type/TypeSignature.java @@ -42,6 +42,7 @@ public class TypeSignature static { BASE_NAME_ALIAS_TO_CANONICAL.put("int", StandardTypes.INTEGER); + BASE_NAME_ALIAS_TO_CANONICAL.put("", "unknown"); } public TypeSignature(String base, TypeSignatureParameter... parameters) diff --git a/presto-spi/src/test/java/com/facebook/presto/spi/type/TestTypeSignature.java b/presto-spi/src/test/java/com/facebook/presto/spi/type/TestTypeSignature.java index 7b33ca0b3c3d..0bcda194ce0b 100644 --- a/presto-spi/src/test/java/com/facebook/presto/spi/type/TestTypeSignature.java +++ b/presto-spi/src/test/java/com/facebook/presto/spi/type/TestTypeSignature.java @@ -80,6 +80,11 @@ public void parseRowSignature() ImmutableSet.of("p1", "s1", "p2", "s2"), rowSignature(namedParameter("a", decimal("p1", "s1")), namedParameter("b", decimal("p2", "s2")))); assertEquals(parseTypeSignature("row(a Int(p1))"), parseTypeSignature("row(a integer(p1))")); + assertRowSignature( + "row()", + "row", + ImmutableList.of("field0 unknown"), + "row(field0 unknown)"); // TODO: remove the following tests when the old style row type has been completely dropped assertOldRowSignature( @@ -177,9 +182,9 @@ public void parseSignature() "map", ImmutableList.of("bigint", "map(bigint,map(varchar,bigint))")); - assertSignatureFail("blah()"); - assertSignatureFail("array()"); - assertSignatureFail("map()"); + assertSignature("blah()", "blah", ImmutableList.of("unknown"), "blah(unknown)"); + assertSignature("array()", "array", ImmutableList.of("unknown"), "array(unknown)"); + assertSignature("map()", "map", ImmutableList.of("unknown"), "map(unknown)"); assertSignatureFail("x", ImmutableSet.of("x")); // ensure this is not treated as a row type From 9d9b6b01444ca07c5df14c9bc638ec840f182efd Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 19 Dec 2017 02:23:12 -0800 Subject: [PATCH 254/331] Add twitter functions --- pom.xml | 1 + presto-twitter-functions/pom.xml | 65 ++++++++++++ .../functions/TwitterFunctionsPlugin.java | 31 ++++++ .../functions/TwitterStringFunctions.java | 99 +++++++++++++++++++ .../functions/TestTwitterFunctions.java | 49 +++++++++ .../src/main/provisio/twitter.xml | 6 ++ 6 files changed, 251 insertions(+) create mode 100644 presto-twitter-functions/pom.xml create mode 100644 presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterFunctionsPlugin.java create mode 100644 presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterStringFunctions.java create mode 100644 presto-twitter-functions/src/test/java/com/facebook/presto/twitter/functions/TestTwitterFunctions.java diff --git a/pom.xml b/pom.xml index bbc8d4251808..43625d6c3f4b 100644 --- a/pom.xml +++ b/pom.xml @@ -118,6 +118,7 @@ presto-thrift-testing-server presto-thrift-connector presto-matching + presto-twitter-functions diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml new file mode 100644 index 000000000000..1e373341fec7 --- /dev/null +++ b/presto-twitter-functions/pom.xml @@ -0,0 +1,65 @@ + + + 4.0.0 + + + com.facebook.presto + presto-root + 0.188-tw-0.43 + + + presto-twitter-functions + Twitter's specific functions for Presto + presto-plugin + + + ${project.parent.basedir} + + + + + com.google.guava + guava + + + + + com.facebook.presto + presto-spi + provided + + + + io.airlift + slice + provided + + + + + org.testng + testng + test + + + + com.facebook.presto + presto-tests + test + + + + com.facebook.presto + presto-main + test + + + + com.facebook.presto + presto-main + test-jar + test + + + + diff --git a/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterFunctionsPlugin.java b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterFunctionsPlugin.java new file mode 100644 index 000000000000..7fceabe9ac82 --- /dev/null +++ b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterFunctionsPlugin.java @@ -0,0 +1,31 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.twitter.functions; + +import com.facebook.presto.spi.Plugin; +import com.google.common.collect.ImmutableSet; + +import java.util.Set; + +public class TwitterFunctionsPlugin + implements Plugin +{ + @Override + public Set> getFunctions() + { + return ImmutableSet.>builder() + .add(TwitterStringFunctions.class) + .build(); + } +} diff --git a/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterStringFunctions.java b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterStringFunctions.java new file mode 100644 index 000000000000..3ad76a156018 --- /dev/null +++ b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterStringFunctions.java @@ -0,0 +1,99 @@ +/* + * 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.twitter.functions; + +import com.facebook.presto.spi.ErrorCodeSupplier; +import com.facebook.presto.spi.PrestoException; +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.function.LiteralParameters; +import com.facebook.presto.spi.function.ScalarFunction; +import com.facebook.presto.spi.function.SqlType; +import com.facebook.presto.spi.type.StandardTypes; +import com.google.common.primitives.Ints; +import io.airlift.slice.Slice; + +import static com.facebook.presto.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; +import static com.facebook.presto.spi.type.VarcharType.VARCHAR; +import static io.airlift.slice.SliceUtf8.offsetOfCodePoint; +import static java.lang.String.format; + +public class TwitterStringFunctions +{ + private TwitterStringFunctions() + { + } + + @ScalarFunction("split_every") + @LiteralParameters({"x"}) + @SqlType("array(varchar(x))") + public static Block str2array(@SqlType("varchar(x)") Slice utf8) + { + return str2array(utf8, 1, utf8.length() + 1); + } + + @ScalarFunction("split_every") + @LiteralParameters({"x"}) + @SqlType("array(varchar(x))") + public static Block str2array(@SqlType("varchar(x)") Slice utf8, @SqlType(StandardTypes.BIGINT) long length) + { + return str2array(utf8, length, utf8.length() / length + 1); + } + + @ScalarFunction("split_every") + @LiteralParameters({"x"}) + @SqlType("array(varchar(x))") + public static Block str2array(@SqlType("varchar(x)") Slice utf8, @SqlType(StandardTypes.BIGINT) long length, @SqlType(StandardTypes.BIGINT) long limit) + { + checkCondition(limit > 0, INVALID_FUNCTION_ARGUMENT, "Limit must be positive"); + checkCondition(limit <= Integer.MAX_VALUE, INVALID_FUNCTION_ARGUMENT, "Limit is too large"); + checkCondition(length > 0, INVALID_FUNCTION_ARGUMENT, "Length must be positive"); + checkCondition(length <= Integer.MAX_VALUE, INVALID_FUNCTION_ARGUMENT, "Length is too large"); + BlockBuilder parts = VARCHAR.createBlockBuilder(new BlockBuilderStatus(), 1, Ints.saturatedCast(length)); + // If limit is one, the last and only element is the complete string + if (limit == 1) { + VARCHAR.writeSlice(parts, utf8); + return parts.build(); + } + + int index = offsetOfCodePoint(utf8, 0); + while (index < utf8.length()) { + int splitIndex = offsetOfCodePoint(utf8, index, Ints.saturatedCast(length)); + // Enough remaining string? + if (splitIndex < 0) { + break; + } + // Add the part from current index to found split + VARCHAR.writeSlice(parts, utf8, index, splitIndex - index); + // Continue after current end + index = splitIndex; + // Reached limit-1 parts so we can stop + if (parts.getPositionCount() == limit - 1) { + break; + } + } + // Rest of string + VARCHAR.writeSlice(parts, utf8, index, utf8.length() - index); + + return parts.build(); + } + + private static void checkCondition(boolean condition, ErrorCodeSupplier errorCode, String formatString, Object... args) + { + if (!condition) { + throw new PrestoException(errorCode, format(formatString, args)); + } + } +} diff --git a/presto-twitter-functions/src/test/java/com/facebook/presto/twitter/functions/TestTwitterFunctions.java b/presto-twitter-functions/src/test/java/com/facebook/presto/twitter/functions/TestTwitterFunctions.java new file mode 100644 index 000000000000..8353d53c7b99 --- /dev/null +++ b/presto-twitter-functions/src/test/java/com/facebook/presto/twitter/functions/TestTwitterFunctions.java @@ -0,0 +1,49 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.twitter.functions; + +import com.facebook.presto.operator.scalar.AbstractTestFunctions; +import com.facebook.presto.spi.type.ArrayType; +import com.google.common.collect.ImmutableList; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static com.facebook.presto.metadata.FunctionExtractor.extractFunctions; +import static com.facebook.presto.spi.type.VarcharType.createVarcharType; + +public class TestTwitterFunctions + extends AbstractTestFunctions +{ + @BeforeClass + public void setUp() + { + functionAssertions.addFunctions(extractFunctions(new TwitterFunctionsPlugin().getFunctions())); + } + + @Test + public void testStr2Array() + { + assertFunction("SPLIT_EVERY('')", new ArrayType(createVarcharType(0)), ImmutableList.of("")); + assertFunction("SPLIT_EVERY('abc')", new ArrayType(createVarcharType(3)), ImmutableList.of("a", "b", "c")); + assertFunction("SPLIT_EVERY('a.b.c')", new ArrayType(createVarcharType(5)), ImmutableList.of("a", ".", "b", ".", "c")); + assertFunction("SPLIT_EVERY('...')", new ArrayType(createVarcharType(3)), ImmutableList.of(".", ".", ".")); + // Test str_to_array for non-ASCII + assertFunction("SPLIT_EVERY('\u4FE1\u5FF5,\u7231,\u5E0C\u671B')", new ArrayType(createVarcharType(7)), ImmutableList.of("\u4FE1", "\u5FF5", ",", "\u7231", ",", "\u5E0C", "\u671B")); + // Test argument length + assertFunction("SPLIT_EVERY('a.b.c', 2)", new ArrayType(createVarcharType(5)), ImmutableList.of("a.", "b.", "c")); + // Test argument limit + assertFunction("SPLIT_EVERY('a.b.c', 2, 1)", new ArrayType(createVarcharType(5)), ImmutableList.of("a.b.c")); + assertFunction("SPLIT_EVERY('a.b.c', 2, 2)", new ArrayType(createVarcharType(5)), ImmutableList.of("a.", "b.c")); + } +} diff --git a/presto-twitter-server/src/main/provisio/twitter.xml b/presto-twitter-server/src/main/provisio/twitter.xml index 9570c58fcfb7..e1da085284c0 100644 --- a/presto-twitter-server/src/main/provisio/twitter.xml +++ b/presto-twitter-server/src/main/provisio/twitter.xml @@ -13,4 +13,10 @@ + + + + + + From 5b4057b5c01a362ef6e2ed31c3978a4f7a9fa834 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 20 Dec 2017 14:28:21 -0800 Subject: [PATCH 255/331] Change version back to 0.188 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 56 files changed, 58 insertions(+), 58 deletions(-) diff --git a/pom.xml b/pom.xml index 43625d6c3f4b..c530b44dea94 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.188-tw-0.43 + 0.188 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index bd39c22d518c..7aff1063715f 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index b8e220e16f9b..f82338b19199 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 092fdbdf6e7b..b2dd0f44c1c2 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index d8b8013517e1..18daf2cd97aa 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 283f7bf2068b..8e74cc381e76 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 0ac504eb6fcd..e17ba6de0e2c 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.188-tw-0.43 + 0.188 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index bdf040f33943..0051682b4800 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 667ef78ec3e8..2550b56517e1 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 3b315bc3ab04..b8fc8e151823 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 6677efaf5215..8ade3e8b43be 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index a4463ddbf9cc..48d22e70392e 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index a1668221505e..ca418f9c2531 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 159a3a95d1f6..3d360add49d8 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 1cacfb08cb11..234d3117d8a1 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-example-http diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index 04519e7bc63d..e7dc3d2a7db2 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 5a12438c8778..3c8520857f14 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 7d818c0941c8..e5d31fbba233 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 8a80f8a97a2a..72d6a4e026ec 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 53c39475b2cb..1a2d5ffdc212 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 033354e90040..98a15af64695 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index d70f592f96e7..2d8bbecbf795 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 733eb044b4f4..9a85dc990fd4 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 1b548a268fb2..9e90907b4a76 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index fe9d970ad613..5e21c2e54dc0 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.188-tw-0.43 + 0.188 presto-matching diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index b92b0b07e1fe..7979b692e009 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 9e65869ed85b..f153bf5d222e 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 58b0d7a47cc5..ccddfd49df20 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index cccade6be601..f9cc2f866afe 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 54fef6e36928..e4dfb550b22e 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 31a7e2d2bd1f..7b86eb659f5a 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 15410679ff07..009ccf666fdc 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 3cac06256774..500d02b5aea7 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index e443c66519fb..cb3c50ebaa94 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.188-tw-0.43 + 0.188 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 48bcae5e99ec..1d8ce69f8922 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index cc05a89ec95d..edae416523ca 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 059f438bed67..4641cff0d36a 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index add057f40286..b9ccf83748f3 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index 155bbaa1800c..5d08e1e45e6d 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index cfca4fe4dc40..6efad51a6816 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 64dd20cb6323..f932e9e1873f 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 82199c74ac35..ed48c674bcf6 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 9419b8408093..d859afadda81 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index f37b3ccfc3ea..49ebc9cd5fbe 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.188-tw-0.43 + 0.188 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 696da318f34e..1a7e56e4c434 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 061dfdab9e5f..008396d7a3e9 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 4b86d165efab..8f2faf81ef38 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.188-tw-0.43 + 0.188 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index 3eb1c188ad72..24bdc1a6de21 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index cec04430c5aa..283c7dd1261d 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index 24fbac9d7fd1..81ff79ce2519 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index 01548b1c0d8c..a2b8488aa31b 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 785f0cc284ba..79beb71cd085 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index 1e373341fec7..c0f5e3a69c41 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-twitter-functions diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index 750183ae9cf7..ef51bf94713e 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 5bba64c85f93..c04927764c7e 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 473b1d6034a4..dfd6981652f4 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.188-tw-0.43 + 0.188 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.188-tw-0.43 + 0.188 provided From 66077cffe9c6ad88318794f34d89503e43a1f074 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 20 Dec 2017 18:07:46 -0800 Subject: [PATCH 256/331] Switch to tw version tag --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial-toolkit/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 57 files changed, 59 insertions(+), 59 deletions(-) diff --git a/pom.xml b/pom.xml index 121e3f98451a..af0f43405328 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.191 + 0.191-tw-0.44 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 40585227dd44..c00433b68029 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index ae85bd468fad..4728ae26ea02 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 47d034678301..c1decb81ff3e 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 42314178190b..07e91874f787 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index af6a97bf57d4..e04bf1b4622f 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 4417d1040f27..a99ab5aea181 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.191 + 0.191-tw-0.44 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index e829779ca04e..b35224090e7a 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 70cdff83e4f1..f6cf78f689ed 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index d9172e9b83bb..823d99fec879 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 9def1899c10c..3a30a9f505c5 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 3e90f9eb84cc..f412b48b43d7 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index cca3c105381e..f378c4cc0e89 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 29d1ecc3bcce..73fb901f36e3 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 7cf1eeccb094..54f135ec4d24 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index c7f70b39a44c..85b70ac1d628 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index 40052f337599..71ff7308f7f9 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 386d61abadea..7c7e90f26884 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index f9400431a61c..c8ebd0c3adf1 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 106af4fe0ba1..2e2fb7a31957 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 99e2cc91915e..923b9ced9ae6 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index a38a6bd0d771..1793681a5ea6 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index 2d8bbecbf795..835bbe27c692 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.191-tw-0.44 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 9894a0bb1ff8..719fe2cae2ca 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 38cf777c3215..589529402d80 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index d9cd7f7921f1..f580bbda814f 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.191 + 0.191-tw-0.44 presto-matching diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index b4c250ca8467..6a88e32b84e3 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 6c295b04e17d..2a95ea165c04 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 2dc7f361fc73..05692ed8bd7e 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 97f1a0798808..eb8076204c13 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index e2e930bd8307..a2a6594bbc86 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index f08d413fd63e..c2a192b6de9e 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 83e776b98034..a31694cd0c34 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index a06785617b4c..31c320e674d0 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 401fb69ba6a7..aa9ffc3e6b0e 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.191 + 0.191-tw-0.44 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index c1792007f91b..0eb430dc4e14 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 7f71bdaeabe8..c8486bd433cd 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 3370a79a35ce..be423ead1031 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 66704a040d4b..728790a9f3d2 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index a494f1806e76..b8d78a01ab51 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 17bbd184eb1c..b4476be2eea7 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 07f0c406558a..2af825461fc7 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index abfcf3dafe62..49eb35565e42 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 5ec449d229e4..293db3630f3c 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index f65a99710a93..4679fe0d1e61 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.191 + 0.191-tw-0.44 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 6b3fb3fd8bbd..40a42628dd75 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 9c3746e26090..d4e1bdb459fd 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index f99236423546..57144a2c57f1 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.191 + 0.191-tw-0.44 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index 4a53c27fcc65..094edea67c85 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 7416bea24765..57698c288f81 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index db52c6f1e779..b551a84e2587 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index 4f97781e6c62..e7afd5f2d641 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 61055d58540b..2f90e3997b6f 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index c0f5e3a69c41..0945e20c34c1 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.191-tw-0.44 presto-twitter-functions diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index ef51bf94713e..834bc657f22b 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.188 + 0.191-tw-0.44 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index cac5bc28cac5..0318d30f90c4 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.191-tw-0.44 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index dfd6981652f4..7b67259c2290 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.188 + 0.191-tw-0.44 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.188 + 0.191-tw-0.44 provided From 6cf45a2f3cf1e149a4c5552bf74d2c53f25fad23 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 21 Dec 2017 02:58:48 -0800 Subject: [PATCH 257/331] Fix issues --- .../presto/hive/BackgroundHiveSplitLoader.java | 1 - .../facebook/presto/hive/HiveClientModule.java | 1 - .../presto/hive/HiveConnectorFactory.java | 4 ---- .../facebook/presto/hive/InternalHiveSplit.java | 3 --- .../metastore/thrift/ThriftMetastoreModule.java | 15 +++++++-------- .../thrift/ThriftHiveRecordCursorProvider.java | 6 ++++-- .../facebook/presto/kafka/util/KafkaLoader.java | 15 ++++++++------- 7 files changed, 19 insertions(+), 26 deletions(-) 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 162efbb21968..a3f358f1b343 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 @@ -508,7 +508,6 @@ private Optional createInternalHiveSplit( boolean forceLocalScheduling = HiveSessionProperties.isForceLocalScheduling(session); - // For empty files, some filesystem (e.g. LocalFileSystem) produce one empty block // while others (e.g. hdfs.DistributedFileSystem) produces no block. // Synthesize an empty block if one does not already exist. 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 427e3120bf09..816d5ee1fb99 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 @@ -26,7 +26,6 @@ import com.facebook.presto.spi.connector.ConnectorPageSourceProvider; import com.facebook.presto.spi.connector.ConnectorSplitManager; import com.facebook.presto.spi.type.TypeManager; -import com.facebook.presto.twitter.hive.PooledHiveMetastoreClientFactory; import com.facebook.presto.twitter.hive.thrift.HiveThriftFieldIdResolverFactory; import com.facebook.presto.twitter.hive.thrift.ThriftFieldIdResolverFactory; import com.facebook.presto.twitter.hive.thrift.ThriftHiveRecordCursorProvider; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveConnectorFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveConnectorFactory.java index def6b58b0b84..bca390b7d088 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveConnectorFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveConnectorFactory.java @@ -32,9 +32,6 @@ import com.facebook.presto.spi.connector.classloader.ClassLoaderSafeConnectorPageSourceProvider; import com.facebook.presto.spi.connector.classloader.ClassLoaderSafeConnectorSplitManager; import com.facebook.presto.spi.connector.classloader.ClassLoaderSafeNodePartitioningProvider; -import com.facebook.presto.twitter.hive.MetastoreStaticClusterModule; -import com.facebook.presto.twitter.hive.MetastoreZkDiscoveryBasedModule; -import com.facebook.presto.twitter.hive.ZookeeperServersetMetastoreConfig; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableSet; import com.google.inject.Injector; @@ -52,7 +49,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Strings.isNullOrEmpty; -import static io.airlift.configuration.ConditionalModule.installModuleIf; import static java.util.Objects.requireNonNull; public class HiveConnectorFactory diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/InternalHiveSplit.java b/presto-hive/src/main/java/com/facebook/presto/hive/InternalHiveSplit.java index 1d66c395cc85..9d776cbb1f1c 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/InternalHiveSplit.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/InternalHiveSplit.java @@ -16,15 +16,12 @@ import com.facebook.presto.spi.HostAddress; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.hadoop.compression.lzo.LzoIndex; -import org.apache.hadoop.fs.Path; import org.openjdk.jol.info.ClassLayout; import javax.annotation.concurrent.NotThreadSafe; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.OptionalInt; import java.util.Properties; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftMetastoreModule.java b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftMetastoreModule.java index a7d59ae4315e..1ba8ea19281d 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftMetastoreModule.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/metastore/thrift/ThriftMetastoreModule.java @@ -21,17 +21,16 @@ import com.facebook.presto.twitter.hive.PooledHiveMetastoreClientFactory; import com.facebook.presto.twitter.hive.ZookeeperServersetMetastoreConfig; import com.google.inject.Binder; -import com.google.inject.Module; import com.google.inject.Scopes; +import io.airlift.configuration.AbstractConfigurationAwareModule; import static io.airlift.configuration.ConditionalModule.installModuleIf; -import static io.airlift.configuration.ConfigBinder.configBinder; import static java.util.Objects.requireNonNull; import static org.weakref.jmx.ObjectNames.generatedNameOf; import static org.weakref.jmx.guice.ExportBinder.newExporter; public class ThriftMetastoreModule - implements Module + extends AbstractConfigurationAwareModule { private final String connectorId; @@ -41,7 +40,7 @@ public ThriftMetastoreModule(String connectorId) } @Override - public void configure(Binder binder) + public void setup(Binder binder) { binder.bind(HiveMetastoreClientFactory.class).in(Scopes.SINGLETON); binder.bind(PooledHiveMetastoreClientFactory.class).in(Scopes.SINGLETON); @@ -58,13 +57,13 @@ public void configure(Binder binder) private void bindMetastoreClusterModule() { - installModuleIf( + install(installModuleIf( ZookeeperServersetMetastoreConfig.class, zkMetastoreConfig -> zkMetastoreConfig.getZookeeperServerHostAndPort() == null, - new MetastoreStaticClusterModule()); - installModuleIf( + new MetastoreStaticClusterModule())); + install(installModuleIf( ZookeeperServersetMetastoreConfig.class, zkMetastoreConfig -> zkMetastoreConfig.getZookeeperServerHostAndPort() != null, - new MetastoreZkDiscoveryBasedModule()); + new MetastoreZkDiscoveryBasedModule())); } } 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 3fab8611c41a..c05dc82aad95 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 @@ -96,12 +96,14 @@ public Optional createRecordCursor( LzoIndex index = new LzoIndex(); try { index = LzoIndex.readIndex(hdfsEnvironment.getFileSystem(session.getUser(), getLzopIndexPath(path), configuration), path); - } catch (IOException ignored) { + } + catch (IOException ignored) { // ignored } if (index.isEmpty()) { length = start == 0 ? fileSize : 0; - } else { + } + else { start = index.alignSliceEndToIndex(start, fileSize); length = index.alignSliceEndToIndex(start + length, fileSize) - start; } diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/KafkaLoader.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/KafkaLoader.java index 9a7af55851bc..f72f650be757 100644 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/KafkaLoader.java +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/KafkaLoader.java @@ -15,7 +15,8 @@ import com.facebook.presto.Session; import com.facebook.presto.client.Column; -import com.facebook.presto.client.QueryResults; +import com.facebook.presto.client.QueryData; +import com.facebook.presto.client.QueryStatusInfo; import com.facebook.presto.server.testing.TestingPrestoServer; import com.facebook.presto.spi.type.TimeZoneKey; import com.facebook.presto.spi.type.Type; @@ -91,16 +92,16 @@ private KafkaLoadingSession(Session session) } @Override - public void addResults(QueryResults results) + public void addResults(QueryStatusInfo statusInfo, QueryData data) { - if (types.get() == null && results.getColumns() != null) { - types.set(getTypes(results.getColumns())); + if (types.get() == null && statusInfo.getColumns() != null) { + types.set(getTypes(statusInfo.getColumns())); } - if (results.getData() != null) { + if (data.getData() != null) { checkState(types.get() != null, "Data without types received!"); - List columns = results.getColumns(); - for (List fields : results.getData()) { + List columns = statusInfo.getColumns(); + for (List fields : data.getData()) { ImmutableMap.Builder builder = ImmutableMap.builder(); for (int i = 0; i < fields.size(); i++) { Type type = types.get().get(i); From 1cef2fe26ae0d7afa48f2899c77dcd20916bdb32 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 21 Dec 2017 07:33:05 -0800 Subject: [PATCH 258/331] Fix lzo index issue --- .../twitter/hive/thrift/ThriftHiveRecordCursor.java | 5 +++++ .../hive/thrift/ThriftHiveRecordCursorProvider.java | 12 +++++++++--- 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java index ade6d392b597..fb558c1e5349 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java @@ -167,6 +167,11 @@ public ThriftHiveRecordCursor( hiveIndexs[i] = column.getHiveColumnIndex(); thriftIds[i] = getThriftIdWithFailOver(thriftFieldIdResolver, hiveIndexs[i]); } + + // close immediately if the totalBytes are zero + if (totalBytes <= 0) { + close(); + } } @Override 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 c05dc82aad95..a5d6a79c9b0c 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 @@ -101,11 +101,17 @@ public Optional createRecordCursor( // ignored } if (index.isEmpty()) { - length = start == 0 ? fileSize : 0; + if (start != 0) { + start = 0; + length = 0; + } + else { + length = fileSize; + } } else { - start = index.alignSliceEndToIndex(start, fileSize); - length = index.alignSliceEndToIndex(start + length, fileSize) - start; + start = index.alignSliceStartToIndex(start, start + length); + length = Math.min(index.alignSliceEndToIndex(start + length, fileSize), fileSize) - start; } } From b0402f17f3160a41fbe2cddcec0f460c13dd563a Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 22 Dec 2017 00:18:33 -0800 Subject: [PATCH 259/331] Improve lzo index issue --- .../hive/thrift/ThriftGeneralInputFormat.java | 25 ++++++++++++++-- .../hive/thrift/ThriftHiveRecordCursor.java | 4 +-- .../ThriftHiveRecordCursorProvider.java | 29 ++++++++++++------- 3 files changed, 44 insertions(+), 14 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java index fea6a8d76875..1f9690036dbf 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGeneralInputFormat.java @@ -18,6 +18,8 @@ import com.twitter.elephantbird.mapreduce.input.MultiInputFormat; import com.twitter.elephantbird.mapreduce.io.BinaryWritable; import com.twitter.elephantbird.util.TypeRef; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.mapred.FileSplit; import org.apache.hadoop.mapred.InputSplit; @@ -29,13 +31,17 @@ import static com.facebook.presto.hive.HiveErrorCode.HIVE_INVALID_METADATA; import static com.facebook.presto.hive.HiveUtil.checkCondition; +import static com.facebook.presto.hive.HiveUtil.getLzopIndexPath; +import static com.facebook.presto.hive.HiveUtil.isLzopCompressedFile; import static java.lang.String.format; import static org.apache.hadoop.hive.serde.Constants.SERIALIZATION_CLASS; /** * Mirror of com.twitter.elephantbird.mapred.input.HiveMultiInputFormat allows to pass the thriftClassName - * directly as a property of JobConfig. - * PR for twitter/elephant-bird: https://github.com/twitter/elephant-bird/pull/481 + * directly as a property of JobConfig and check lzo index existence when check splitability. + * PR for twitter/elephant-bird: + * https://github.com/twitter/elephant-bird/pull/481 + * https://github.com/twitter/elephant-bird/pull/485 * Remove the class once #481 is included in a release */ @SuppressWarnings("deprecation") @@ -61,6 +67,21 @@ private void initialize(FileSplit split, JobConf job) throws IOException } } + @Override + public boolean isSplitable(FileSystem fs, Path filename) + { + if (isLzopCompressedFile(filename)) { + Path indexFile = getLzopIndexPath(filename); + try { + return fs.exists(indexFile); + } + catch (IOException e) { + return false; + } + } + return super.isSplitable(fs, filename); + } + @Override public RecordReader getRecordReader( InputSplit split, diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java index fb558c1e5349..6cd1ba00443f 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java @@ -168,8 +168,8 @@ public ThriftHiveRecordCursor( thriftIds[i] = getThriftIdWithFailOver(thriftFieldIdResolver, hiveIndexs[i]); } - // close immediately if the totalBytes are zero - if (totalBytes <= 0) { + // close immediately if the number of totalBytes is zero + if (totalBytes == 0) { close(); } } 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 a5d6a79c9b0c..8c10e78f5f33 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 @@ -100,18 +100,27 @@ public Optional createRecordCursor( catch (IOException ignored) { // ignored } - if (index.isEmpty()) { - if (start != 0) { - start = 0; - length = 0; - } - else { - length = fileSize; - } + + // re-align split start + if (index.isEmpty() && start != 0) { + // empty index and split not start from beginning + start = LzoIndex.NOT_FOUND; } - else { + if (!index.isEmpty()) { + // align start based on index start = index.alignSliceStartToIndex(start, start + length); - length = Math.min(index.alignSliceEndToIndex(start + length, fileSize), fileSize) - start; + } + + // re-align split end + if (start == LzoIndex.NOT_FOUND) { + // split start cannot be find, then we should skip this split + start = 0; + length = 0; + } + else { + // if index is empty but start is not NOT_FOUND, read the whole file, otherwise align split end. + length = index.isEmpty() ? fileSize : + (Math.min(index.alignSliceEndToIndex(start + length, fileSize), fileSize) - start); } } From 7aa438baf0ea350ebad6f69acd20bde408dab062 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 4 Jan 2018 22:24:26 -0800 Subject: [PATCH 260/331] Allow non-secured request authentication over pathes matching given regrex This is useful when the non-secured port hosted behide a secured front-end service and direct non-secured port access has been limited by a firewall --- .../server/security/AuthenticationFilter.java | 9 ++++++--- .../presto/server/security/SecurityConfig.java | 16 ++++++++++++++++ .../server/security/ServerSecurityModule.java | 1 + .../server/security/TestSecurityConfig.java | 7 +++++-- 4 files changed, 28 insertions(+), 5 deletions(-) diff --git a/presto-main/src/main/java/com/facebook/presto/server/security/AuthenticationFilter.java b/presto-main/src/main/java/com/facebook/presto/server/security/AuthenticationFilter.java index 1c644cfb9cc6..f4f312acfb23 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/security/AuthenticationFilter.java +++ b/presto-main/src/main/java/com/facebook/presto/server/security/AuthenticationFilter.java @@ -36,17 +36,20 @@ import static com.google.common.io.ByteStreams.copy; import static com.google.common.io.ByteStreams.nullOutputStream; import static com.google.common.net.HttpHeaders.WWW_AUTHENTICATE; +import static java.util.Objects.requireNonNull; import static javax.servlet.http.HttpServletResponse.SC_UNAUTHORIZED; public class AuthenticationFilter implements Filter { private final Set authenticators; + private final String httpAuthenticationPathRegex; @Inject - public AuthenticationFilter(Set authenticators) + public AuthenticationFilter(Set authenticators, SecurityConfig securityConfig) { this.authenticators = ImmutableSet.copyOf(authenticators); + this.httpAuthenticationPathRegex = requireNonNull(securityConfig.getHttpAuthenticationPathRegex(), "httpAuthenticationPathRegex is null"); } @Override @@ -62,8 +65,8 @@ public void doFilter(ServletRequest servletRequest, ServletResponse servletRespo HttpServletRequest request = (HttpServletRequest) servletRequest; HttpServletResponse response = (HttpServletResponse) servletResponse; - // skip authentication if non-secure or not configured - if (!request.isSecure() || authenticators.isEmpty()) { + // skip authentication if (not configured) or (non-secure and not match httpAuthenticationPathRegex) + if (authenticators.isEmpty() || (!request.isSecure() && !request.getPathInfo().matches(httpAuthenticationPathRegex))) { nextFilter.doFilter(request, response); return; } diff --git a/presto-main/src/main/java/com/facebook/presto/server/security/SecurityConfig.java b/presto-main/src/main/java/com/facebook/presto/server/security/SecurityConfig.java index ed3003108fee..6492c8a65584 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/security/SecurityConfig.java +++ b/presto-main/src/main/java/com/facebook/presto/server/security/SecurityConfig.java @@ -33,6 +33,8 @@ public class SecurityConfig private Set authenticationTypes = ImmutableSet.of(); + private String httpAuthenticationPathRegex = "^\b$"; + public enum AuthenticationType { CERTIFICATE, @@ -66,4 +68,18 @@ public SecurityConfig setAuthenticationTypes(String types) .collect(toImmutableSet()); return this; } + + @NotNull + public String getHttpAuthenticationPathRegex() + { + return httpAuthenticationPathRegex; + } + + @Config("http-server.http.authentication.path.regex") + @ConfigDescription("Regex of path that needs to be authenticated for non-secured http request") + public SecurityConfig setHttpAuthenticationPathRegex(String regex) + { + httpAuthenticationPathRegex = regex; + return this; + } } diff --git a/presto-main/src/main/java/com/facebook/presto/server/security/ServerSecurityModule.java b/presto-main/src/main/java/com/facebook/presto/server/security/ServerSecurityModule.java index 87c60836db3d..fafc5c02c127 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/security/ServerSecurityModule.java +++ b/presto-main/src/main/java/com/facebook/presto/server/security/ServerSecurityModule.java @@ -38,6 +38,7 @@ protected void setup(Binder binder) { newSetBinder(binder, Filter.class, TheServlet.class).addBinding() .to(AuthenticationFilter.class).in(Scopes.SINGLETON); + configBinder(binder).bindConfig(SecurityConfig.class); Set authTypes = buildConfigObject(SecurityConfig.class).getAuthenticationTypes(); Multibinder authBinder = newSetBinder(binder, Authenticator.class); diff --git a/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java b/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java index 4da645dfcad9..11d56ad00e37 100644 --- a/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java +++ b/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java @@ -29,7 +29,8 @@ public class TestSecurityConfig public void testDefaults() { ConfigAssertions.assertRecordedDefaults(ConfigAssertions.recordDefaults(SecurityConfig.class) - .setAuthenticationTypes("")); + .setAuthenticationTypes("") + .setHttpAuthenticationPathRegex("^\b$")); } @Test @@ -37,10 +38,12 @@ public void testExplicitPropertyMappings() { Map properties = new ImmutableMap.Builder() .put("http-server.authentication.type", "KERBEROS,LDAP") + .put("http-server.http.authentication.path.regex", "^/v1/statement") .build(); SecurityConfig expected = new SecurityConfig() - .setAuthenticationTypes(ImmutableSet.of(KERBEROS, LDAP)); + .setAuthenticationTypes(ImmutableSet.of(KERBEROS, LDAP)) + .setHttpAuthenticationPathRegex("^/v1/statement"); ConfigAssertions.assertFullMapping(properties, expected); } From f738bedabc1e78e9bc059e15f0ccc9f4a65dd77c Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 18 Jan 2018 00:43:09 -0800 Subject: [PATCH 261/331] Revert twitter version to oss version --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial-toolkit/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 57 files changed, 59 insertions(+), 59 deletions(-) diff --git a/pom.xml b/pom.xml index af0f43405328..121e3f98451a 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.191-tw-0.44 + 0.191 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index c00433b68029..40585227dd44 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 4728ae26ea02..ae85bd468fad 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index c1decb81ff3e..47d034678301 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 07e91874f787..42314178190b 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index e04bf1b4622f..af6a97bf57d4 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index a99ab5aea181..4417d1040f27 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.191-tw-0.44 + 0.191 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index b35224090e7a..e829779ca04e 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index f6cf78f689ed..70cdff83e4f1 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 823d99fec879..d9172e9b83bb 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 3a30a9f505c5..9def1899c10c 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index f412b48b43d7..3e90f9eb84cc 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index f378c4cc0e89..cca3c105381e 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 73fb901f36e3..29d1ecc3bcce 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 54f135ec4d24..7cf1eeccb094 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index 85b70ac1d628..c7f70b39a44c 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index 71ff7308f7f9..40052f337599 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 7c7e90f26884..386d61abadea 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index c8ebd0c3adf1..f9400431a61c 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 2e2fb7a31957..106af4fe0ba1 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 923b9ced9ae6..99e2cc91915e 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 1793681a5ea6..a38a6bd0d771 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index 835bbe27c692..01ddeb2e197d 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 719fe2cae2ca..9894a0bb1ff8 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 589529402d80..38cf777c3215 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index f580bbda814f..d9cd7f7921f1 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.191-tw-0.44 + 0.191 presto-matching diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 6a88e32b84e3..b4c250ca8467 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 2a95ea165c04..6c295b04e17d 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 05692ed8bd7e..2dc7f361fc73 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index eb8076204c13..97f1a0798808 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index a2a6594bbc86..e2e930bd8307 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index c2a192b6de9e..f08d413fd63e 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index a31694cd0c34..83e776b98034 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 31c320e674d0..a06785617b4c 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index aa9ffc3e6b0e..401fb69ba6a7 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.191-tw-0.44 + 0.191 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 0eb430dc4e14..c1792007f91b 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index c8486bd433cd..7f71bdaeabe8 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index be423ead1031..3370a79a35ce 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 728790a9f3d2..66704a040d4b 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index b8d78a01ab51..a494f1806e76 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index b4476be2eea7..17bbd184eb1c 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 2af825461fc7..07f0c406558a 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 49eb35565e42..abfcf3dafe62 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 293db3630f3c..5ec449d229e4 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 4679fe0d1e61..f65a99710a93 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.191-tw-0.44 + 0.191 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 40a42628dd75..6b3fb3fd8bbd 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index d4e1bdb459fd..9c3746e26090 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 57144a2c57f1..f99236423546 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.191-tw-0.44 + 0.191 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index 094edea67c85..4a53c27fcc65 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 57698c288f81..7416bea24765 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index b551a84e2587..db52c6f1e779 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index e7afd5f2d641..4f97781e6c62 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 2f90e3997b6f..61055d58540b 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index 0945e20c34c1..ad16b1a8c185 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-twitter-functions diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index 834bc657f22b..debf86b7ed2a 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 0318d30f90c4..cac5bc28cac5 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 7b67259c2290..95c7729cbdca 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191-tw-0.44 + 0.191 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.191-tw-0.44 + 0.191 provided From df2ba31350185c49ca1b0bdfb80c28ec4e1899df Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 18 Jan 2018 01:11:33 -0800 Subject: [PATCH 262/331] Upgrade to 0.192-tw-0.45 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial-toolkit/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- .../com/facebook/presto/hive/HiveCoercionPolicy.java | 1 - .../java/com/facebook/presto/hive/HivePageSource.java | 1 - .../twitter/hive/PooledHiveMetastoreClientFactory.java | 1 - .../presto/twitter/hive/ZookeeperMetastoreMonitor.java | 1 - .../presto/twitter/hive/thrift/ThriftGenericRow.java | 6 +++--- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- .../java/com/facebook/presto/kafka/KafkaRecordSet.java | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- .../com/facebook/presto/tests/hive/TestHiveCoercion.java | 9 +++++---- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- .../facebook/presto/decoder/thrift/ThriftGenericRow.java | 6 +++--- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- .../plugin/eventlistener/QueryCompletedEventScriber.java | 2 -- .../presto/plugin/eventlistener/QueryStatsHelper.java | 2 -- .../plugin/eventlistener/TwitterEventListener.java | 1 - .../plugin/eventlistener/TwitterEventListenerPlugin.java | 1 - .../presto/plugin/eventlistener/TwitterScriber.java | 1 - 70 files changed, 71 insertions(+), 81 deletions(-) diff --git a/pom.xml b/pom.xml index e9694e3cdddb..e40a10e15b9a 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.192 + 0.192-tw-0.45 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index ec5eb453054c..6a4d64181176 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index dd4e989fafa2..77484e248212 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index f74507b66feb..de2d1b110efc 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 71d1ce2a131b..5c1b0a58c519 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 41c32ab681a8..e3ec3a1bd746 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 3729c44f534c..0860d2bc2dd2 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.192 + 0.192-tw-0.45 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index cb226fe3cbba..2a0393d6f3aa 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 8d5cc181516f..2832f8eee4ab 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index b9aa465fe048..a34475dcb71c 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 3129b71e7663..4ee256fd129f 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 06cb719b06b3..e2b1e22c16bd 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index c38f681c2c8e..2ad4847012e4 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index d28e207b0b7f..3a5fcab7c8e4 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index e6dc6d613a5e..4f4582864cc9 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index 20d57dfb05de..9d6a993a3b06 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index fbb2766d7bd5..eb1c5f226d92 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 05a404235f7a..81cd875667f8 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index c00b0af40cb2..484ba270ef37 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-hive diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveCoercionPolicy.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveCoercionPolicy.java index 7224b35cc791..9ee10ff12b7c 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveCoercionPolicy.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveCoercionPolicy.java @@ -32,7 +32,6 @@ import static com.facebook.presto.hive.HiveType.HIVE_INT; import static com.facebook.presto.hive.HiveType.HIVE_LONG; import static com.facebook.presto.hive.HiveType.HIVE_SHORT; - import static java.util.Objects.requireNonNull; public class HiveCoercionPolicy diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSource.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSource.java index aad02ee7d4ef..12ebca2758bb 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSource.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSource.java @@ -35,7 +35,6 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.Arrays; - import java.util.List; import java.util.function.Function; import java.util.stream.Collectors; diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java index bcb6e7acaa82..270e313e7511 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/PooledHiveMetastoreClientFactory.java @@ -14,7 +14,6 @@ package com.facebook.presto.twitter.hive; import com.facebook.presto.hive.HiveClientConfig; - import com.facebook.presto.hive.authentication.HiveMetastoreAuthentication; import com.facebook.presto.hive.metastore.thrift.HiveMetastoreClient; import com.facebook.presto.hive.metastore.thrift.ThriftHiveMetastoreClient; diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperMetastoreMonitor.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperMetastoreMonitor.java index f340abf47334..cf8df021c1a1 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperMetastoreMonitor.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperMetastoreMonitor.java @@ -14,7 +14,6 @@ package com.facebook.presto.twitter.hive; import com.google.common.net.HostAndPort; - import io.airlift.log.Logger; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java index c319bba2d375..5b961f7d813a 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftGenericRow.java @@ -42,9 +42,9 @@ public class ThriftGenericRow { private static final Logger log = Logger.get(ThriftGenericRow.class); private final Map values = new HashMap<>(); - private byte[] buf = null; - private int off = 0; - private int len = 0; + private byte[] buf; + private int off; + private int len; public ThriftGenericRow() { diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 2b7b4e6ba7cc..45fca52ec159 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index e97234e646f4..09ed9d97c0d7 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 565b7a54ae65..1dbf5c3a76ac 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index 01ddeb2e197d..b9aa8a5e50e4 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.192-tw-0.45 presto-kafka07 diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java index 8b2eb7303db5..34283d277cef 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java @@ -125,7 +125,7 @@ public class KafkaRecordCursor private long totalMessages; private long cursorOffset = split.getStart(); private Iterator messageAndOffsetIterator; - private long fetchedSize = 0L; + private long fetchedSize; private final AtomicBoolean reported = new AtomicBoolean(); private final long startTs; diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index e6c4af208da4..37dd068b11fb 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 3fd9f827a101..b53a5026aba2 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index 053bf68d2bad..08186f11c77b 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.192 + 0.192-tw-0.45 presto-matching diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index ea12fca96f2b..983127ac2db5 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 6e3938a24fb8..9d3791c289d7 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 02a434a2a8f9..a75e1ac164f0 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 30dc1c374a80..da53f50b43f4 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 9b70968fbb48..dca307a3b3d5 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 83faece07eaa..3068f2bdf638 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index a0da082e3e72..441027f279b1 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index c717d2cec25b..10e5f6b473dd 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 0224f0f12b2f..8be0df8d95c2 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.192 + 0.192-tw-0.45 presto-product-tests diff --git a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveCoercion.java b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveCoercion.java index c0ac04402da4..5d8a6da7db7f 100644 --- a/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveCoercion.java +++ b/presto-product-tests/src/main/java/com/facebook/presto/tests/hive/TestHiveCoercion.java @@ -144,10 +144,11 @@ private static String getCreateDummyTableDDL(HiveTableDefinition tableDefinition String tableName = mutableTableInstanceOf(tableDefinition).getNameInDatabase(); String floatToDoubleType = tableName.toLowerCase(Locale.ENGLISH).contains("parquet") ? "DOUBLE" : "FLOAT"; return tableDefinition.getCreateTableDDL(format(dummyTableNameFormat, tableName), Optional.empty()) - .replace(format(" float_to_double %s,", floatToDoubleType), format(" float_to_double %s", floatToDoubleType)) - .replace(format(" row_to_row STRUCT < tinyint_to_smallint : TINYINT, tinyint_to_int : TINYINT, tinyint_to_bigint : TINYINT, smallint_to_int : SMALLINT, smallint_to_bigint : SMALLINT, int_to_bigint : INT, bigint_to_varchar : BIGINT, float_to_double : %s >,", floatToDoubleType), "") - .replace(format(" list_to_list ARRAY < STRUCT < tinyint_to_smallint : TINYINT, tinyint_to_int : TINYINT, tinyint_to_bigint : TINYINT, smallint_to_int : SMALLINT, smallint_to_bigint : SMALLINT, int_to_bigint : INT, bigint_to_varchar : BIGINT, float_to_double : %s > >,", floatToDoubleType), "") - .replace(format(" map_to_map MAP < TINYINT, STRUCT < tinyint_to_smallint : TINYINT, tinyint_to_int : TINYINT, tinyint_to_bigint : TINYINT, smallint_to_int : SMALLINT, smallint_to_bigint : SMALLINT, int_to_bigint : INT, bigint_to_varchar : BIGINT, float_to_double : %s > >", floatToDoubleType), ""); + .replace(format(" float_to_double %s,", floatToDoubleType), format(" float_to_double %s", floatToDoubleType)) + .replace(format(" row_to_row STRUCT < tinyint_to_smallint : TINYINT, tinyint_to_int : TINYINT, tinyint_to_bigint : TINYINT, smallint_to_int : SMALLINT, smallint_to_bigint : SMALLINT, int_to_bigint : INT, bigint_to_varchar : BIGINT, float_to_double : %s >,", floatToDoubleType), "") + .replace(format(" list_to_list ARRAY < STRUCT < tinyint_to_smallint : TINYINT, tinyint_to_int : TINYINT, tinyint_to_bigint : TINYINT, smallint_to_int : SMALLINT, smallint_to_bigint : SMALLINT, int_to_bigint : INT, bigint_to_varchar : BIGINT, float_to_double : %s > >,", floatToDoubleType), "") + .replace(format(" map_to_map MAP < TINYINT, STRUCT < tinyint_to_smallint : TINYINT, tinyint_to_int : TINYINT, tinyint_to_bigint : TINYINT, smallint_to_int : SMALLINT, smallint_to_bigint : SMALLINT, int_to_bigint : INT, bigint_to_varchar : BIGINT, float_to_double : %s > >", floatToDoubleType), ""); + } private static HiveTableDefinition.HiveTableDefinitionBuilder avroTableDefinitionBuilder() { diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 78d840d9faff..9bdfed765aea 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 98ad8c554f62..444b1a99a93f 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index d88ad5729542..a43789103be5 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-record-decoder diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftGenericRow.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftGenericRow.java index 53e25894583a..f3f356983bed 100644 --- a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftGenericRow.java +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftGenericRow.java @@ -42,9 +42,9 @@ public class ThriftGenericRow { private static final Logger log = Logger.get(ThriftGenericRow.class); private final Map values = new HashMap<>(); - private byte[] buf = null; - private int off = 0; - private int len = 0; + private byte[] buf; + private int off; + private int len; public ThriftGenericRow() { diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index d4909611ee3a..48e97c3212a0 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index fd5b0efaf384..fb92ae3c1607 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 3186fd47ae5d..62752c1b1226 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 8016af3475ac..37b3fdd531ac 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index f829fd5740a9..7994a596edc8 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 3ea95495bb51..17999f482b0a 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index e3ae149b8d5d..b20183fb90e1 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.192 + 0.192-tw-0.45 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 3bb5288e552c..ae4d4c523bf0 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 706532adb34c..4bed6bb8178c 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 9e549ff06903..7025587713e7 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.192 + 0.192-tw-0.45 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index e0364d59ad27..ea2abc1a7ce4 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 5d38c83fae57..09498b4b407b 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index dfbb99fb8bb4..b36af9208b94 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index 814a41c1688e..0b7b8aacc5a8 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index c07e75635f15..1c6f5df03d5a 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index ad16b1a8c185..bcd83115d811 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.192-tw-0.45 presto-twitter-functions diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index debf86b7ed2a..9bc5a5f701cd 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.191 + 0.192-tw-0.45 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index ade0c334dba1..6815d312c1c4 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192 + 0.192-tw-0.45 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 95c7729cbdca..7ff5fb8d1f29 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.191 + 0.192-tw-0.45 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.191 + 0.192-tw-0.45 provided diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java index 4a9376fa7290..477d3e302355 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java @@ -18,10 +18,8 @@ import com.facebook.presto.spi.eventlistener.QueryFailureInfo; import com.facebook.presto.spi.eventlistener.QueryMetadata; import com.facebook.presto.spi.eventlistener.QueryStatistics; - import com.twitter.presto.thriftjava.QueryCompletionEvent; import com.twitter.presto.thriftjava.QueryState; - import io.airlift.log.Logger; import org.apache.thrift.TException; diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java index 62cb231075b0..e57cedda3ff0 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java @@ -15,10 +15,8 @@ import com.facebook.presto.spi.eventlistener.QueryMetadata; import com.facebook.presto.spi.eventlistener.QueryStatistics; - import com.twitter.presto.thriftjava.OperatorStats; import com.twitter.presto.thriftjava.QueryStageInfo; - import io.airlift.log.Logger; import io.airlift.units.DataSize; import io.airlift.units.Duration; diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java index ef9e0ed2b36c..6df4eb29c54b 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java @@ -17,7 +17,6 @@ import com.facebook.presto.spi.eventlistener.QueryCompletedEvent; import com.facebook.presto.spi.eventlistener.QueryCreatedEvent; import com.facebook.presto.spi.eventlistener.SplitCompletedEvent; - import io.airlift.log.Logger; public class TwitterEventListener diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerPlugin.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerPlugin.java index 5caf7b8724a9..1b77b1decce7 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerPlugin.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerPlugin.java @@ -15,7 +15,6 @@ import com.facebook.presto.spi.Plugin; import com.facebook.presto.spi.eventlistener.EventListenerFactory; - import com.google.common.collect.ImmutableList; public class TwitterEventListenerPlugin diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java index 8acc7ca0ad9e..8f4088da6aeb 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java @@ -17,7 +17,6 @@ import com.twitter.logging.Level; import com.twitter.logging.QueueingHandler; import com.twitter.logging.ScribeHandler; - import org.apache.thrift.TBase; import org.apache.thrift.TException; import org.apache.thrift.TSerializer; From 5cf9713dc064993ab8e902c1e7d076b3d2f9098c Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 25 Jan 2018 19:29:19 -0800 Subject: [PATCH 263/331] Ignore not found files --- .../com/facebook/presto/hive/util/HiveFileIterator.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/util/HiveFileIterator.java b/presto-hive/src/main/java/com/facebook/presto/hive/util/HiveFileIterator.java index 94205aaa3ec9..6a2610930888 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/util/HiveFileIterator.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/util/HiveFileIterator.java @@ -98,7 +98,14 @@ protected LocatedFileStatus computeNext() if (paths.isEmpty()) { return endOfData(); } - remoteIterator = getLocatedFileStatusRemoteIterator(paths.removeFirst()); + try { + remoteIterator = getLocatedFileStatusRemoteIterator(paths.removeFirst()); + } + catch (PrestoException e) { + if (!e.getErrorCode().equals(HIVE_FILE_NOT_FOUND.toErrorCode())) { + throw e; + } + } } } From 0c005cde1cf6f768f1b1ae7db45b1597d87b242d Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 26 Jan 2018 01:29:28 -0800 Subject: [PATCH 264/331] Release 0.192-tw-0.46 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial-toolkit/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 57 files changed, 59 insertions(+), 59 deletions(-) diff --git a/pom.xml b/pom.xml index e40a10e15b9a..0fe8e0ed0e4c 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.192-tw-0.45 + 0.192-tw-0.46 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 6a4d64181176..0477e631d65e 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 77484e248212..9c3b457474ee 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index de2d1b110efc..a2810b55669b 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 5c1b0a58c519..b26c058fa8ab 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index e3ec3a1bd746..d88bae968090 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 0860d2bc2dd2..decf6dfe415a 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.192-tw-0.45 + 0.192-tw-0.46 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 2a0393d6f3aa..93f52ba09c10 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 2832f8eee4ab..fd0fee6eb14e 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index a34475dcb71c..22c0b1fc75d9 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 4ee256fd129f..fa33f9e645eb 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index e2b1e22c16bd..40a82d378518 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 2ad4847012e4..2197ed89bba1 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 3a5fcab7c8e4..8e389fe6fd0b 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 4f4582864cc9..a84a56898ad2 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index 9d6a993a3b06..ea38c10b8563 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index eb1c5f226d92..05c79fb41aad 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 81cd875667f8..adbbcd47ceb3 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 484ba270ef37..e22ca0ff948e 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 45fca52ec159..0f04b1dfea05 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 09ed9d97c0d7..db8dbf1548b2 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 1dbf5c3a76ac..77c12e09a26e 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index b9aa8a5e50e4..557c10655afe 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 37dd068b11fb..396601366fba 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index b53a5026aba2..203f6a8accec 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index 08186f11c77b..be80ddcc81b6 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.192-tw-0.45 + 0.192-tw-0.46 presto-matching diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 983127ac2db5..39b0e3dad4e9 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 9d3791c289d7..fab77909fb7b 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index a75e1ac164f0..574cf423d37f 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index da53f50b43f4..fcffa7c2ace0 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index dca307a3b3d5..f78c7bf82a55 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 3068f2bdf638..c0e37f3a7e2b 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 441027f279b1..7fb868de7188 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 10e5f6b473dd..2b27988d19a1 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 8be0df8d95c2..8b4689e16970 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.192-tw-0.45 + 0.192-tw-0.46 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 9bdfed765aea..c88bc368e87d 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 444b1a99a93f..4d8fe7f161bc 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index a43789103be5..b9c8b16eac2c 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 48e97c3212a0..d52a138d069d 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index fb92ae3c1607..cb579f13763f 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 62752c1b1226..2850148ba857 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 37b3fdd531ac..bd2e2a04a29a 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 7994a596edc8..0f6ebb5237c0 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 17999f482b0a..f1ed4682fdf6 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index b20183fb90e1..58e2b26452f0 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.192-tw-0.45 + 0.192-tw-0.46 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index ae4d4c523bf0..0248604caead 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 4bed6bb8178c..3664ebe375f5 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 7025587713e7..641f91f6fb10 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.192-tw-0.45 + 0.192-tw-0.46 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index ea2abc1a7ce4..7e477f9b7267 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 09498b4b407b..706794ca737e 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index b36af9208b94..338ae7d1eece 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index 0b7b8aacc5a8..6619e09f4f45 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 1c6f5df03d5a..20a1930f7546 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index bcd83115d811..78885905b816 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-twitter-functions diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index 9bc5a5f701cd..a65a0c9ca3d1 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 6815d312c1c4..8d1d88df1668 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 7ff5fb8d1f29..5775c5942c31 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.45 + 0.192-tw-0.46 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.192-tw-0.45 + 0.192-tw-0.46 provided From 7285a68febcdfe768acc3e93d034ae6e0cb2cb30 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 13 Feb 2018 13:39:48 -0800 Subject: [PATCH 265/331] Update travis notification token due to twitter slack group name change (#137) --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 7ca8bfaa3bc3..2a89d2c825d9 100644 --- a/.travis.yml +++ b/.travis.yml @@ -138,7 +138,7 @@ notifications: rooms: secure: peNh1KxwlxIpFyb60S8AMvaJThgh1LsjE+Whf1rYkJalVd2wUrqBIoyDKVSueyHD01hQ06gT7rBV6Pu/QcBMR1a9BbMCjERfxLZFUAheuC2Rsb+p1c4dyvBcFUGacgW7XWKCaVYGDGxuUvb0I3Z8cR6KxhK2xi88tHiqBGVGV2yI6zzOTpWVknMfFBtn+ONU1Ob2P6trclXaDyFd4MxubULri6CQdl35eQAq/VnmR3SZOgyVu3V30MGKwI3zhSli+3VqmW0JmaDGoHN6gznM1+VqABLgmIq0P+n+r5gdZWRCorq10NZCFMhVQ8U6rQHcL7sAniYJJsC/yRt6+pjyzIF4N+LSzZ7T+FLxQqT7k/1ukNgrujLDfTpn76Mo9eYTZmfAdzbm1QKJDACwr8Slqhq1jGzcrFMHunvXhVqjOs24R+JAHblY0O9PXvv7aR29GOQWDCvD7nV5QBUr8Xz5q7ozbLqHTI+yH02Jj4EaZ+azWYdRmnr9wDBxWMYBEgOdj4pII9b298XEDB72TxA3KpLTpdLxBTR+gIk/LjJqb/wb84xUv8gPXkaXccltGd5YI90c84cX8isbzNkAylzyfF2Eyueh0XbnMHfpFqBS7qaVM0/D+UxZkU0WNJ0x7G9XJvkiq49bZz2q1KLE4XuvVnTZSSjVSUAS8RtHfwUV33c= slack: - secure: R0kQ9zwPGx2wKCiiB8crxmniDdYuWMAzkbYKbMMICmIksI+H/15JLXkx8z9DrE0EyjivqhXSGl+CThKmlJKkwEZCpGxVPd2N+kEmYWr/wfjJzfZ/ZqfS6t4+WyCUIIdNiOJM63cjK+0wEH0HPYa//LIFC+DPcvowOBVHTzoUVySXlo83E5mW/1ZvYd4BsVLuI3u01ZkonPVIdtmEErykexPqKExBvpna7homryJTZKF3lUUKsH8GYaDZYF0QdF1F0srxqgAj+fiU16CsXoN2drmDlzK++fB0ilHUW7fhmLju/zfse1k6m7w5/2JBFp0m0WsPo3qOEJBCjyujIwqisQVU90IrRKofMMdOUrElDyruBNU49OatTjQN97YnOLCkNY3fmSPepZb+xhBtn5Flhk2cT8i127rMWaWdpGS/T9IPY2c+/zyXjZDF1F2JTQ1ofKBLQ3jNnXmk0QLVcxj+wFRFSCABZlvsUGOIyayZHfsVojYvvhxG7wpYCU+dc5HtNxCaoloPcPtMD02fgKbdogeRUT/Rkbf8ta74BzoH0XcQx8zSQFN+Dz8xEowBHQq+juWVcH2IIjE08x0Aob4MMyAXuFxagR/WfTDKDpcHSE1SME4D8304fWj91M66ll4Sz89sRw2iIBHGG4dB/WWeE4r3gF4ciRFRYJCr/rTeCJY= + secure: E7XVlbdwIdKxnr6Tk1rmCefufs1w8h4nCWz79Uh6wMma8gC7x5ChKFqwvLRJ0WUpmPS+Ng1xeTv+wmb8TMDv2X8snmht9420/TFRy9wi1aLWNJXQUveNBzn83sCS40jFi6gd9xqKawd68R84UVH3PeNhksDtDnKAblx71miwbKmLwHc1KFoLMEnaaWEg5NgFl8/UadYDvsLD44v6YDza8eYrLp3aGK8v9ewBDySHE16IHAfpteTRaU0kG/H1kvVvFdH/h/sSPfimehd51b4i3mm/nRrjJ/VSLc7p9w5FkHUECtA0N6zcytRxN6MrbhrxJ8XG3vte3KSRSFCqfgOSRM2NWcca4CtBP2V0SwrAYMo5jim6fr921lfcbUTWTSnvMYLC17QrAxoclVrgK05GjGoLgSH42UPGf3QNkqXzyueNzaLJ+KSlgwFblIQKp6WGZYSRorL0F7s50pIoqMVoebcrnB0ObK/CcE2ywS/HeTgoSkWSDSmKBsO+cmtv1yAamy9DlmgRGZlxIxdBELXtHRkQ2B6Z2QdiQU4MHiFBc/IESJbnCait4odn+oJUjehZg+b9vjCoWwVw3zNMIJhokyxO8SiyKJmbO0z1g2L/BykWGI1DQu8HkeQzO+CmNUV3AOrxDG3amL/tkB/06fyQtnYMDhUhvX64uWSaE36sYL4= before_deploy: - mkdir /tmp/artifacts From 37bdeb7b2ea43ef3a192f324022a56b487c22ab6 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 13 Feb 2018 13:40:36 -0800 Subject: [PATCH 266/331] Allow giving a complete service principle for Kerberos authentication (#136) The built-in host-based service name type for Kerberos authenticator cannot fit our use case. By allowing pass a specific service principle and use the user based name type, the authenticator can be authenticated by Kerberos master without specifying the host running Presto service. --- .../main/java/com/facebook/presto/client/SpnegoHandler.java | 6 ++++-- .../presto/server/security/KerberosAuthenticator.java | 5 +++-- .../conf/presto/etc/singlenode-kerberized.properties | 2 +- .../tempto/tempto-configuration-for-docker-kerberos.yaml | 2 +- 4 files changed, 9 insertions(+), 6 deletions(-) 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 index cb08cb598392..bf8b33484c34 100644 --- a/presto-client/src/main/java/com/facebook/presto/client/SpnegoHandler.java +++ b/presto-client/src/main/java/com/facebook/presto/client/SpnegoHandler.java @@ -75,6 +75,7 @@ public class SpnegoHandler private static final Oid KERBEROS_OID = createOid("1.2.840.113554.1.2.2"); private final String remoteServiceName; + private final boolean isCompleteServicePrinciple; private final boolean useCanonicalHostname; private final Optional principal; private final Optional keytab; @@ -92,6 +93,7 @@ public SpnegoHandler( Optional credentialCache) { this.remoteServiceName = requireNonNull(remoteServiceName, "remoteServiceName is null"); + this.isCompleteServicePrinciple = remoteServiceName.contains("@"); this.useCanonicalHostname = useCanonicalHostname; this.principal = requireNonNull(principal, "principal is null"); this.keytab = requireNonNull(keytab, "keytab is null"); @@ -134,7 +136,7 @@ private static boolean isNegotiate(String value) private Request authenticate(Request request) { String hostName = request.url().host(); - String principal = makeServicePrincipal(remoteServiceName, hostName, useCanonicalHostname); + String principal = isCompleteServicePrinciple ? remoteServiceName : makeServicePrincipal(remoteServiceName, hostName, useCanonicalHostname); byte[] token = generateToken(principal); String credential = format("%s %s", NEGOTIATE, Base64.getEncoder().encodeToString(token)); @@ -150,7 +152,7 @@ private byte[] generateToken(String servicePrincipal) Session session = getSession(); context = doAs(session.getLoginContext().getSubject(), () -> { GSSContext result = GSS_MANAGER.createContext( - GSS_MANAGER.createName(servicePrincipal, NT_HOSTBASED_SERVICE), + GSS_MANAGER.createName(servicePrincipal, isCompleteServicePrinciple ? NT_USER_NAME : NT_HOSTBASED_SERVICE), SPNEGO_OID, session.getClientCredential(), INDEFINITE_LIFETIME); diff --git a/presto-main/src/main/java/com/facebook/presto/server/security/KerberosAuthenticator.java b/presto-main/src/main/java/com/facebook/presto/server/security/KerberosAuthenticator.java index af9e7211ad2d..6f1f7cae41ce 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/security/KerberosAuthenticator.java +++ b/presto-main/src/main/java/com/facebook/presto/server/security/KerberosAuthenticator.java @@ -64,8 +64,9 @@ public KerberosAuthenticator(KerberosConfig config) System.setProperty("java.security.krb5.conf", config.getKerberosConfig().getAbsolutePath()); try { + boolean isCompleteServicePrinciple = config.getServiceName().contains("@"); String hostname = InetAddress.getLocalHost().getCanonicalHostName().toLowerCase(Locale.US); - String servicePrincipal = config.getServiceName() + "/" + hostname; + String servicePrincipal = isCompleteServicePrinciple ? config.getServiceName() : config.getServiceName() + "/" + hostname; loginContext = new LoginContext("", null, null, new Configuration() { @Override @@ -91,7 +92,7 @@ public AppConfigurationEntry[] getAppConfigurationEntry(String name) loginContext.login(); serverCredential = doAs(loginContext.getSubject(), () -> gssManager.createCredential( - gssManager.createName(config.getServiceName() + "@" + hostname, GSSName.NT_HOSTBASED_SERVICE), + isCompleteServicePrinciple ? gssManager.createName(config.getServiceName(), GSSName.NT_USER_NAME) : gssManager.createName(config.getServiceName() + "@" + hostname, GSSName.NT_HOSTBASED_SERVICE), INDEFINITE_LIFETIME, new Oid[] { new Oid("1.2.840.113554.1.2.2"), // kerberos 5 diff --git a/presto-product-tests/conf/presto/etc/singlenode-kerberized.properties b/presto-product-tests/conf/presto/etc/singlenode-kerberized.properties index b012c831a5a7..43ea74e648db 100644 --- a/presto-product-tests/conf/presto/etc/singlenode-kerberized.properties +++ b/presto-product-tests/conf/presto/etc/singlenode-kerberized.properties @@ -18,7 +18,7 @@ discovery.uri=https://presto-master.docker.cluster:7778 http.authentication.krb5.config=/etc/krb5.conf http-server.authentication.type=KERBEROS,CERTIFICATE -http.server.authentication.krb5.service-name=presto-server +http.server.authentication.krb5.service-name=presto-server/presto-master.docker.cluster@LABS.TERADATA.COM http-server.http.enabled=false http-server.https.enabled=true http-server.https.port=7778 diff --git a/presto-product-tests/conf/tempto/tempto-configuration-for-docker-kerberos.yaml b/presto-product-tests/conf/tempto/tempto-configuration-for-docker-kerberos.yaml index 54e2d063630d..16ef4773805b 100644 --- a/presto-product-tests/conf/tempto/tempto-configuration-for-docker-kerberos.yaml +++ b/presto-product-tests/conf/tempto/tempto-configuration-for-docker-kerberos.yaml @@ -37,7 +37,7 @@ databases: cli_kerberos_principal: presto-client/presto-master.docker.cluster@LABS.TERADATA.COM cli_kerberos_keytab: /etc/presto/conf/presto-client.keytab cli_kerberos_config_path: /etc/krb5.conf - cli_kerberos_service_name: presto-server + cli_kerberos_service_name: presto-server/presto-master.docker.cluster@LABS.TERADATA.COM cli_kerberos_use_canonical_hostname: false configured_hdfs_user: hdfs From d09410796d9eecb4a94f15e82884bf08cd5b6c6a Mon Sep 17 00:00:00 2001 From: Hao Luo Date: Fri, 16 Feb 2018 15:49:47 -0800 Subject: [PATCH 267/331] Use HadoopLzoDecompressor in ParquetCompressionUtils --- .../hive/parquet/HadoopLzoDecompressor.java | 69 +++++++++++++++++++ .../hive/parquet/ParquetCompressionUtils.java | 3 +- 2 files changed, 70 insertions(+), 2 deletions(-) create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/parquet/HadoopLzoDecompressor.java diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/HadoopLzoDecompressor.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/HadoopLzoDecompressor.java new file mode 100644 index 000000000000..a87b648ca14b --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/HadoopLzoDecompressor.java @@ -0,0 +1,69 @@ +/* + * 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.parquet; + +import com.hadoop.compression.lzo.LzoCodec; +import io.airlift.compress.Decompressor; +import io.airlift.compress.MalformedInputException; +import org.apache.hadoop.conf.Configuration; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import static com.google.common.base.Preconditions.checkArgument; + +// HadoopLzoDecompressor from aircompressor +public class HadoopLzoDecompressor + implements Decompressor +{ + private static final Configuration HADOOP_CONF = new Configuration(); + + private final org.apache.hadoop.io.compress.Decompressor decompressor; + + public HadoopLzoDecompressor() + { + LzoCodec codec = new LzoCodec(); + codec.setConf(HADOOP_CONF); + decompressor = codec.createDecompressor(); + } + + @Override + public int decompress(byte[] input, int inputOffset, int inputLength, byte[] output, int outputOffset, int maxOutputLength) + throws MalformedInputException + { + decompressor.reset(); + decompressor.setInput(input, inputOffset, inputLength); + + int offset = outputOffset; + int outputLimit = outputOffset + maxOutputLength; + while (!decompressor.finished() && offset < outputLimit) { + try { + offset += decompressor.decompress(output, offset, outputLimit - offset); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + checkArgument(decompressor.getRemaining() == 0); + return offset - outputOffset; + } + + @Override + public void decompress(ByteBuffer input, ByteBuffer output) + throws MalformedInputException + { + throw new UnsupportedOperationException("Not supported in HadoopLzoDecompressor"); + } +} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetCompressionUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetCompressionUtils.java index ef9a300a4acd..963f432e408b 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetCompressionUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetCompressionUtils.java @@ -14,7 +14,6 @@ package com.facebook.presto.hive.parquet; import io.airlift.compress.Decompressor; -import io.airlift.compress.lzo.LzoDecompressor; import io.airlift.compress.snappy.SnappyDecompressor; import io.airlift.slice.DynamicSliceOutput; import io.airlift.slice.Slice; @@ -88,7 +87,7 @@ private static Slice decompressGzip(Slice input, int uncompressedSize) private static Slice decompressLZO(Slice input, int uncompressedSize) { - LzoDecompressor lzoDecompressor = new LzoDecompressor(); + Decompressor lzoDecompressor = new HadoopLzoDecompressor(); long totalDecompressedCount = 0; // over allocate buffer which makes decompression easier byte[] output = new byte[uncompressedSize + SIZE_OF_LONG]; From fc8c262197a8e9b1069bdd5c38f607efa2973c02 Mon Sep 17 00:00:00 2001 From: Hao Luo Date: Tue, 20 Feb 2018 13:06:01 -0800 Subject: [PATCH 268/331] Release 0.192-tw-0.47 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial-toolkit/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 57 files changed, 59 insertions(+), 59 deletions(-) diff --git a/pom.xml b/pom.xml index 0fe8e0ed0e4c..68657a2f39b8 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.192-tw-0.46 + 0.192-tw-0.47 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 0477e631d65e..36a3684c8603 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 9c3b457474ee..1873d6338531 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index a2810b55669b..062b43b6bec0 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index b26c058fa8ab..7de36823493f 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index d88bae968090..0056e9ccfa2b 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index decf6dfe415a..22b99c884ccc 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.192-tw-0.46 + 0.192-tw-0.47 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 93f52ba09c10..b4c4604176a4 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index fd0fee6eb14e..6358189e0d20 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 22c0b1fc75d9..9c521e2d2411 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index fa33f9e645eb..7d66e0d8e92f 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 40a82d378518..ae70dff7ed3d 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 2197ed89bba1..3f32e36be33e 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 8e389fe6fd0b..6dd0bd01ed24 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index a84a56898ad2..fc82ae48fc13 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index ea38c10b8563..07684a3b483c 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index 05c79fb41aad..075162abbbfb 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index adbbcd47ceb3..c787da9e4ab1 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index e22ca0ff948e..7717350766f4 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 0f04b1dfea05..f8b42d93ba43 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index db8dbf1548b2..9556bd98dff2 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 77c12e09a26e..973a273cedf2 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index 557c10655afe..30f48025e8cd 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 396601366fba..40cd30596c15 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 203f6a8accec..9a8352636b06 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index be80ddcc81b6..6286cc055868 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.192-tw-0.46 + 0.192-tw-0.47 presto-matching diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 39b0e3dad4e9..c1b15c11a18a 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index fab77909fb7b..d329a893c8c6 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 574cf423d37f..25da6daaca53 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index fcffa7c2ace0..9fb7409c55d2 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index f78c7bf82a55..4c6f7a3f7db9 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index c0e37f3a7e2b..3ce2dad9e4fe 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 7fb868de7188..9c4784f3fa34 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 2b27988d19a1..768277d16ebb 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 8b4689e16970..0d7cdade5677 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.192-tw-0.46 + 0.192-tw-0.47 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index c88bc368e87d..9cb9905fd9f2 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 4d8fe7f161bc..853b9884ef3b 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index b9c8b16eac2c..04f8b2530423 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index d52a138d069d..ee20cc57d297 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index cb579f13763f..03414870d5be 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 2850148ba857..a6fecbb21fec 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index bd2e2a04a29a..a1752e150c8b 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 0f6ebb5237c0..623df1d7b8b3 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index f1ed4682fdf6..ddc2798fd4da 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 58e2b26452f0..7ba297748d70 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.192-tw-0.46 + 0.192-tw-0.47 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 0248604caead..17b18183c06d 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 3664ebe375f5..87fe3e9f35d3 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 641f91f6fb10..5040a337ac38 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.192-tw-0.46 + 0.192-tw-0.47 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index 7e477f9b7267..300cfd91f4b1 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 706794ca737e..75658100f900 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index 338ae7d1eece..9150c42d4d4f 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index 6619e09f4f45..73be719cbf39 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 20a1930f7546..160669298e7c 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index 78885905b816..65774cbb88ce 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-twitter-functions diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index a65a0c9ca3d1..a4b3787bf761 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 8d1d88df1668..2684ba89fa68 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 5775c5942c31..89350a868038 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.46 + 0.192-tw-0.47 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.192-tw-0.46 + 0.192-tw-0.47 provided From 6b7fe25ac3fe7b75c59be776e8652ca4a83f19af Mon Sep 17 00:00:00 2001 From: Hao Luo Date: Thu, 22 Feb 2018 17:53:54 -0800 Subject: [PATCH 269/331] Fix find field by name in optimized parquet reader --- .../presto/hive/parquet/ParquetPageSource.java | 4 ++-- .../presto/hive/parquet/ParquetTypeUtils.java | 15 +++++++++++++++ 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSource.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSource.java index 91ad75f2028a..0b885cbd6ea9 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSource.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSource.java @@ -40,8 +40,8 @@ import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveErrorCode.HIVE_BAD_DATA; import static com.facebook.presto.hive.HiveErrorCode.HIVE_CURSOR_ERROR; +import static com.facebook.presto.hive.parquet.ParquetTypeUtils.findFieldIndexByName; import static com.facebook.presto.hive.parquet.ParquetTypeUtils.getDescriptor; -import static com.facebook.presto.hive.parquet.ParquetTypeUtils.getFieldIndex; import static com.facebook.presto.hive.parquet.ParquetTypeUtils.getParquetType; import static com.facebook.presto.spi.type.StandardTypes.ARRAY; import static com.facebook.presto.spi.type.StandardTypes.MAP; @@ -170,7 +170,7 @@ public Page getNextPage() Type type = types.get(fieldId); int fieldIndex; if (useParquetColumnNames) { - fieldIndex = getFieldIndex(fileSchema, columnNames.get(fieldId)); + fieldIndex = findFieldIndexByName(fileSchema, columnNames.get(fieldId)); } else { fieldIndex = hiveColumnIndexes[fieldId]; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java index 3bd6d241db9c..3bf89764040b 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java @@ -169,6 +169,21 @@ private static parquet.schema.Type findParquetTypeByName(HiveColumnHandle column return type; } + // Find the column index by name following the same logic as findParquetTypeByName + public static int findFieldIndexByName(MessageType fileSchema, String name) + { + // direct match and case-insensitive match + int fieldIndex = getFieldIndex(fileSchema, name); + + // when a parquet field is a hive keyword we append an _ to it in hive. + // try remove _ and direct match / case-insensitive match again + if (fieldIndex == -1 && name.endsWith("_")) { + fieldIndex = getFieldIndex(fileSchema, name.substring(0, name.length() - 1)); + } + + return fieldIndex; + } + public static ParquetEncoding getParquetEncoding(Encoding encoding) { switch (encoding) { From d9c32ba9a8687d0a7b64aaa3dcf8a23d76a91b6b Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 23 Feb 2018 15:19:18 -0800 Subject: [PATCH 270/331] Release 0.192-tw-0.48 (#144) --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial-toolkit/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 57 files changed, 59 insertions(+), 59 deletions(-) diff --git a/pom.xml b/pom.xml index 68657a2f39b8..e5617327b8fa 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.192-tw-0.47 + 0.192-tw-0.48 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 36a3684c8603..6629ead483d0 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 1873d6338531..50864bc80ac2 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 062b43b6bec0..08c0ba1dc8b9 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 7de36823493f..0017b1241672 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 0056e9ccfa2b..c6b84e7b8ea5 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 22b99c884ccc..2117e4f4b13e 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.192-tw-0.47 + 0.192-tw-0.48 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index b4c4604176a4..2970fef6deaf 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 6358189e0d20..0f7616e121f0 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index 9c521e2d2411..a7e3d361d966 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 7d66e0d8e92f..6c2ae9eb6212 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index ae70dff7ed3d..f98d744c4d6c 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 3f32e36be33e..dc83da864915 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 6dd0bd01ed24..3b7928b175a6 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index fc82ae48fc13..2ca606022304 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index 07684a3b483c..00de702404df 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index 075162abbbfb..ae99810e8436 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index c787da9e4ab1..1467cde8440c 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 7717350766f4..74c716f02735 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index f8b42d93ba43..b8603cbcc0ea 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 9556bd98dff2..f5ee2e494014 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 973a273cedf2..d86d027d985a 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index 30f48025e8cd..72adecf4c82e 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 40cd30596c15..fbafd4fd3071 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 9a8352636b06..ac755445408c 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index 6286cc055868..1edde93cceb0 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.192-tw-0.47 + 0.192-tw-0.48 presto-matching diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index c1b15c11a18a..1a034cee2518 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index d329a893c8c6..200af4ec8689 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 25da6daaca53..bc47d14769d3 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 9fb7409c55d2..d68bf94167bd 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 4c6f7a3f7db9..a6e40716c654 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 3ce2dad9e4fe..b93324e85747 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 9c4784f3fa34..24f36f08b875 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 768277d16ebb..c4c5bb8dc6ac 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 0d7cdade5677..f8d8d3e43318 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.192-tw-0.47 + 0.192-tw-0.48 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 9cb9905fd9f2..667670b470ea 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 853b9884ef3b..19fdfd549855 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 04f8b2530423..f41e43ee3858 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index ee20cc57d297..1a5c9b0c5662 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index 03414870d5be..c681c640ce3f 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index a6fecbb21fec..ae93198c3171 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index a1752e150c8b..a8d0642602e0 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 623df1d7b8b3..2fff2d4d1b1b 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index ddc2798fd4da..ab6974f6b1ed 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 7ba297748d70..73ddfa64a273 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.192-tw-0.47 + 0.192-tw-0.48 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 17b18183c06d..f0e67e9ecde6 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 87fe3e9f35d3..d8032faf1fc8 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 5040a337ac38..136c5e6f2eb3 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.192-tw-0.47 + 0.192-tw-0.48 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index 300cfd91f4b1..3dc7d144f367 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 75658100f900..052be12dabc2 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index 9150c42d4d4f..41badf664228 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index 73be719cbf39..2a0c71d24ee7 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 160669298e7c..c1099b5f39f1 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index 65774cbb88ce..2d5d38a3c467 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-twitter-functions diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index a4b3787bf761..d93b20cc9b2e 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 2684ba89fa68..6bf3a44ae69d 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 89350a868038..b60e6f7eb183 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.47 + 0.192-tw-0.48 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.192-tw-0.47 + 0.192-tw-0.48 provided From 912b22dd18b71006fb0447cc1c8247c08644e5c2 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 20 Mar 2018 19:15:09 -0700 Subject: [PATCH 271/331] Add Snowflake functions MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Add utility functions for snowflake ids (https://developer.twitter.com/en/docs/basics/twitter-ids). is_snowflake(id) → boolean Return if a bigint is a snowflake ID (true/false). first_snowflake_for(timestamp) → bigint Return the first snowflake ID given a timestamp. timestamp_from_snowflake(id) → timestamp Return the timestamp given a snowflake ID. cluster_id_from_snowflake(id) → bigint Return the cluster ID given a snowflake ID. instance_id_from_snowflake(id) → bigint Return the instance ID given a snowflake ID. sequence_num_from_snowflake(id) → bigint Return the sequence number given a snowflake ID. --- presto-docs/src/main/sphinx/functions.rst | 1 + .../src/main/sphinx/functions/twitter.rst | 34 +++++ .../twitter/functions/SnowflakeFunctions.java | 129 ++++++++++++++++++ .../functions/TwitterFunctionsPlugin.java | 1 + .../functions/TestTwitterFunctions.java | 33 +++++ 5 files changed, 198 insertions(+) create mode 100644 presto-docs/src/main/sphinx/functions/twitter.rst create mode 100644 presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/SnowflakeFunctions.java diff --git a/presto-docs/src/main/sphinx/functions.rst b/presto-docs/src/main/sphinx/functions.rst index 2abc6e2e45f9..068bb6f86d57 100644 --- a/presto-docs/src/main/sphinx/functions.rst +++ b/presto-docs/src/main/sphinx/functions.rst @@ -26,3 +26,4 @@ Functions and Operators functions/geospatial functions/color functions/teradata + functions/twitter diff --git a/presto-docs/src/main/sphinx/functions/twitter.rst b/presto-docs/src/main/sphinx/functions/twitter.rst new file mode 100644 index 000000000000..e14f1bb6ebf2 --- /dev/null +++ b/presto-docs/src/main/sphinx/functions/twitter.rst @@ -0,0 +1,34 @@ +================== +Twitter Functions +================== + +These functions provide some convenience functionality commonly used at Twitter. + +Twitter IDs(Snowflake) Functions +--------------------------------- + +The utility functions for `Twitter IDs(Snowflake) `_. + +.. function:: is_snowflake(id) -> boolean + + Return if a bigint is a snowflake ID (true/false). + +.. function:: first_snowflake_for(timestamp) -> bigint + + Return the first snowflake ID given a timestamp. + +.. function:: timestamp_from_snowflake(id) -> timestamp + + Return the timestamp given a snowflake ID. + +.. function:: cluster_id_from_snowflake(id) -> bigint + + Return the cluster ID given a snowflake ID. + +.. function:: instance_id_from_snowflake(id) -> bigint + + Return the instance ID given a snowflake ID. + +.. function:: sequence_num_from_snowflake(id) -> bigint + + Return the sequence number given a snowflake ID. diff --git a/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/SnowflakeFunctions.java b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/SnowflakeFunctions.java new file mode 100644 index 000000000000..979cb95a8504 --- /dev/null +++ b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/SnowflakeFunctions.java @@ -0,0 +1,129 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.facebook.presto.twitter.functions; + +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.function.Description; +import com.facebook.presto.spi.function.ScalarFunction; +import com.facebook.presto.spi.function.SqlType; +import com.facebook.presto.spi.type.StandardTypes; + +import java.util.concurrent.TimeUnit; + +import static com.facebook.presto.spi.StandardErrorCode.INVALID_FUNCTION_ARGUMENT; + +/* size in + * bits: 1 41 5 5 12 + * +-+-----------------------------------------+-----+-----+------------+ + * |0| milliseconds |clstr|instc| sequence | + * |0| since twepoch | id | id | number | + * +-+-----------------------------------------+-----+-----+------------+ + * | + * +- Most significant bit + */ +public class SnowflakeFunctions +{ + private static final long SequenceNumBits = 12L; + private static final long MaxSequenceNum = (1L << SequenceNumBits) - 1; + private static final long SequenceNumMask = MaxSequenceNum; + + private static final long InstanceIdBits = 5L; + private static final long MaxInstanceId = (1L << InstanceIdBits) - 1; + private static final long InstanceIdShift = SequenceNumBits; + private static final long InstanceIdMask = MaxInstanceId << InstanceIdShift; + + private static final long ClusterIdBits = 5L; + private static final long MaxClusterId = (1L << ClusterIdBits) - 1; + private static final long ClusterIdShift = InstanceIdShift + InstanceIdBits; + private static final long ClusterIdMask = MaxClusterId << ClusterIdShift; + + private static final long TimestampBits = 41L; + private static final long MaxTimestamp = (1L << TimestampBits) - 1; + private static final long TimestampShift = ClusterIdShift + ClusterIdBits; + private static final long TimestampMask = MaxTimestamp << TimestampShift; + + /* Twepoch is 2010-11-04T01:42:54Z. + * Value is in millis since Unix Epoch 1970-01-01T00:00:00Z. + */ + private static final long Twepoch = 1288834974657L; + private static final long FirstSnowflakeIdUnixTime = Twepoch + TimeUnit.DAYS.toMillis(1); // 1 day after Twepoch. + private static final long FirstSnowflakeId = firstSnowflakeIdFor(FirstSnowflakeIdUnixTime); + + private SnowflakeFunctions() + { + } + + @ScalarFunction("is_snowflake") + @Description("Check if a BIGINT is a Snowflake ID") + @SqlType(StandardTypes.BOOLEAN) + public static boolean isSnowflakeId(@SqlType(StandardTypes.BIGINT) long id) + { + return id >= FirstSnowflakeId; + } + + @ScalarFunction("first_snowflake_for") + @Description("Return the first snowflake ID given a timestamp") + @SqlType(StandardTypes.BIGINT) + public static long firstSnowflakeIdFor(@SqlType(StandardTypes.TIMESTAMP) long timestamp) + { + if (timestamp < FirstSnowflakeIdUnixTime) { + throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "Invalid UnixTimeMillis: UnixTimeMillis[" + timestamp + "] >= FirstSnowflakeIdUnixTime"); + } + return ((timestamp - Twepoch) << TimestampShift); + } + + @ScalarFunction("timestamp_from_snowflake") + @Description("Return the timestamp given a snowflake ID") + @SqlType(StandardTypes.TIMESTAMP) + public static long timestampFromSnowflakeId(@SqlType(StandardTypes.BIGINT) long id) + { + if (!isSnowflakeId(id)) { + throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "Not a Snowflake Id: " + id); + } + return ((id & TimestampMask) >> TimestampShift) + Twepoch; + } + + @ScalarFunction("cluster_id_from_snowflake") + @Description("Return the cluster id given a snowflake ID") + @SqlType(StandardTypes.BIGINT) + public static long clusterIdFromSnowflakeId(@SqlType(StandardTypes.BIGINT) long id) + { + if (!isSnowflakeId(id)) { + throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "Not a Snowflake Id: " + id); + } + return (id & ClusterIdMask) >> ClusterIdShift; + } + + @ScalarFunction("instance_id_from_snowflake") + @Description("Return the instance id given a snowflake ID") + @SqlType(StandardTypes.BIGINT) + public static long instanceIdFromSnowflakeId(@SqlType(StandardTypes.BIGINT) long id) + { + if (!isSnowflakeId(id)) { + throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "Not a Snowflake Id: " + id); + } + return (id & InstanceIdMask) >> InstanceIdShift; + } + + @ScalarFunction("sequence_num_from_snowflake") + @Description("Return the sequence number given a snowflake ID") + @SqlType(StandardTypes.BIGINT) + public static long sequenceNumFromSnowflakeId(@SqlType(StandardTypes.BIGINT) long id) + { + if (!isSnowflakeId(id)) { + throw new PrestoException(INVALID_FUNCTION_ARGUMENT, "Not a Snowflake Id: " + id); + } + return id & SequenceNumMask; + } +} diff --git a/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterFunctionsPlugin.java b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterFunctionsPlugin.java index 7fceabe9ac82..6a9a491e9306 100644 --- a/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterFunctionsPlugin.java +++ b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterFunctionsPlugin.java @@ -26,6 +26,7 @@ public Set> getFunctions() { return ImmutableSet.>builder() .add(TwitterStringFunctions.class) + .add(SnowflakeFunctions.class) .build(); } } diff --git a/presto-twitter-functions/src/test/java/com/facebook/presto/twitter/functions/TestTwitterFunctions.java b/presto-twitter-functions/src/test/java/com/facebook/presto/twitter/functions/TestTwitterFunctions.java index 8353d53c7b99..4c75f59a88f5 100644 --- a/presto-twitter-functions/src/test/java/com/facebook/presto/twitter/functions/TestTwitterFunctions.java +++ b/presto-twitter-functions/src/test/java/com/facebook/presto/twitter/functions/TestTwitterFunctions.java @@ -15,11 +15,16 @@ import com.facebook.presto.operator.scalar.AbstractTestFunctions; import com.facebook.presto.spi.type.ArrayType; +import com.facebook.presto.spi.type.SqlTimestamp; import com.google.common.collect.ImmutableList; import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; import static com.facebook.presto.metadata.FunctionExtractor.extractFunctions; +import static com.facebook.presto.spi.type.BigintType.BIGINT; +import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; +import static com.facebook.presto.spi.type.TimeZoneKey.UTC_KEY; +import static com.facebook.presto.spi.type.TimestampType.TIMESTAMP; import static com.facebook.presto.spi.type.VarcharType.createVarcharType; public class TestTwitterFunctions @@ -46,4 +51,32 @@ public void testStr2Array() assertFunction("SPLIT_EVERY('a.b.c', 2, 1)", new ArrayType(createVarcharType(5)), ImmutableList.of("a.b.c")); assertFunction("SPLIT_EVERY('a.b.c', 2, 2)", new ArrayType(createVarcharType(5)), ImmutableList.of("a.", "b.c")); } + + private static SqlTimestamp toTimestampUTC(long millis) + { + return new SqlTimestamp(millis, UTC_KEY); + } + + @Test + public void testSnowflake() + { + assertFunction("IS_SNOWFLAKE(1000)", BOOLEAN, false); + assertFunction("IS_SNOWFLAKE(265605588183052288)", BOOLEAN, true); + assertFunction("IS_SNOWFLAKE(-265605588183052288)", BOOLEAN, false); + + assertFunction("FIRST_SNOWFLAKE_FOR(from_unixtime(1352160281.593))", BIGINT, 265605588182892544L); + assertInvalidFunction("FIRST_SNOWFLAKE_FOR(from_unixtime(1000))", "Invalid UnixTimeMillis: UnixTimeMillis[1000000] >= FirstSnowflakeIdUnixTime"); + + assertFunction("TIMESTAMP_FROM_SNOWFLAKE(265605588183052288)", TIMESTAMP, toTimestampUTC(1352160281593L)); + assertInvalidFunction("TIMESTAMP_FROM_SNOWFLAKE(1000)", "Not a Snowflake Id: 1000"); + + assertFunction("CLUSTER_ID_FROM_SNOWFLAKE(265605588183052288)", BIGINT, 1L); + assertInvalidFunction("CLUSTER_ID_FROM_SNOWFLAKE(1000)", "Not a Snowflake Id: 1000"); + + assertFunction("INSTANCE_ID_FROM_SNOWFLAKE(265605588183052288)", BIGINT, 7L); + assertInvalidFunction("INSTANCE_ID_FROM_SNOWFLAKE(1000)", "Not a Snowflake Id: 1000"); + + assertFunction("SEQUENCE_NUM_FROM_SNOWFLAKE(265605588183052288)", BIGINT, 0L); + assertInvalidFunction("SEQUENCE_NUM_FROM_SNOWFLAKE(1000)", "Not a Snowflake Id: 1000"); + } } From 55fe33e92a73cf4a30f29114743155bf1698b839 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 23 Mar 2018 11:19:18 -0700 Subject: [PATCH 272/331] Add documents for split_every functions --- .../src/main/sphinx/functions/twitter.rst | 17 +++++++++++++++++ .../functions/TwitterStringFunctions.java | 4 ++++ 2 files changed, 21 insertions(+) diff --git a/presto-docs/src/main/sphinx/functions/twitter.rst b/presto-docs/src/main/sphinx/functions/twitter.rst index e14f1bb6ebf2..03bec3f9247d 100644 --- a/presto-docs/src/main/sphinx/functions/twitter.rst +++ b/presto-docs/src/main/sphinx/functions/twitter.rst @@ -4,6 +4,23 @@ Twitter Functions These functions provide some convenience functionality commonly used at Twitter. +String Functions +----------------- + +.. function:: split_every(string) -> array + + Splits ``string`` on every character and returns an array. + +.. function:: split_every(string, length) -> array + + Splits ``string`` on every ``length`` characters and returns an array. + length must be a positive number. + +.. function:: split_every(string, length, limit) -> array + + Splits ``string`` on every ``length`` characters and returns an array of size at most ``limit``. + The last element in the array always contains everything left in the string. ``limit`` must be a positive number. + Twitter IDs(Snowflake) Functions --------------------------------- diff --git a/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterStringFunctions.java b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterStringFunctions.java index 3ad76a156018..dac9cf846a9a 100644 --- a/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterStringFunctions.java +++ b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterStringFunctions.java @@ -18,6 +18,7 @@ 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.function.Description; import com.facebook.presto.spi.function.LiteralParameters; import com.facebook.presto.spi.function.ScalarFunction; import com.facebook.presto.spi.function.SqlType; @@ -37,6 +38,7 @@ private TwitterStringFunctions() } @ScalarFunction("split_every") + @Description("Splits the string on every character and returns an array") @LiteralParameters({"x"}) @SqlType("array(varchar(x))") public static Block str2array(@SqlType("varchar(x)") Slice utf8) @@ -45,6 +47,7 @@ public static Block str2array(@SqlType("varchar(x)") Slice utf8) } @ScalarFunction("split_every") + @Description("Splits the string on every given length of characters and returns an array") @LiteralParameters({"x"}) @SqlType("array(varchar(x))") public static Block str2array(@SqlType("varchar(x)") Slice utf8, @SqlType(StandardTypes.BIGINT) long length) @@ -53,6 +56,7 @@ public static Block str2array(@SqlType("varchar(x)") Slice utf8, @SqlType(Standa } @ScalarFunction("split_every") + @Description("Splits the string on every given length of characters and returns an array with the size at most of the given limit") @LiteralParameters({"x"}) @SqlType("array(varchar(x))") public static Block str2array(@SqlType("varchar(x)") Slice utf8, @SqlType(StandardTypes.BIGINT) long length, @SqlType(StandardTypes.BIGINT) long limit) From 8f84b9b1f529a418f28af810ccbf6d6f17f36252 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 23 Mar 2018 18:19:20 -0700 Subject: [PATCH 273/331] Move version to new oss version --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-bytecode/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial-toolkit/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 57 files changed, 59 insertions(+), 59 deletions(-) diff --git a/pom.xml b/pom.xml index e5617327b8fa..7e28d9447d3f 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.192-tw-0.48 + 0.196 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 6629ead483d0..bff7399ac8ae 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 50864bc80ac2..61f7c47f284b 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 08c0ba1dc8b9..8bf490655dff 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 0017b1241672..bed1c09824e4 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index c6b84e7b8ea5..a1211ba8ef4e 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 2117e4f4b13e..a008359b6794 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.192-tw-0.48 + 0.196 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 2970fef6deaf..3ad7af423377 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 0f7616e121f0..09d55354885c 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-blackhole diff --git a/presto-bytecode/pom.xml b/presto-bytecode/pom.xml index a7e3d361d966..26a21a941796 100644 --- a/presto-bytecode/pom.xml +++ b/presto-bytecode/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-bytecode diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 6c2ae9eb6212..79593f8e48e5 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index f98d744c4d6c..e69efe6b9fdb 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index dc83da864915..ad519caaa992 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 3b7928b175a6..c2ce1665324d 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 2ca606022304..f77d1888a579 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index 00de702404df..5ea31873c3c2 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index ae99810e8436..c3ee3eaba84b 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 1467cde8440c..5394b686b579 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 74c716f02735..744f90f64b79 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index b8603cbcc0ea..da80bbeaf83f 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index f5ee2e494014..1905b875133e 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index d86d027d985a..640f0ad39c59 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index 72adecf4c82e..825e616468b0 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index fbafd4fd3071..ecaf297fd20f 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index ac755445408c..dd569b3a93df 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index 1edde93cceb0..068761506a79 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.192-tw-0.48 + 0.196 presto-matching diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 1a034cee2518..7b9ce0472e59 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 200af4ec8689..1aed1cfd810d 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index bc47d14769d3..f4f610700932 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index d68bf94167bd..197d01fac3f3 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index a6e40716c654..3876f0cc956a 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index b93324e85747..c2c8cc6b4dca 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-parser diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 24f36f08b875..39826b6f3a8a 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index c4c5bb8dc6ac..56bac316f97e 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index f8d8d3e43318..749080a18d95 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.192-tw-0.48 + 0.196 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 667670b470ea..f29bcee2224a 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 19fdfd549855..13b32deb7141 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index f41e43ee3858..ba7c8ccb3ab3 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 1a5c9b0c5662..3421dbaf44bb 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index c681c640ce3f..ab7dd6f956d1 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index ae93198c3171..c2d59a78ecc3 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index a8d0642602e0..7b59da2a926c 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 2fff2d4d1b1b..ebd88abecf0e 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index ab6974f6b1ed..ca5e0f8ba489 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 73ddfa64a273..39e57a4f08e0 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.192-tw-0.48 + 0.196 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index f0e67e9ecde6..db25c6e73fdd 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index d8032faf1fc8..c7298620b767 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 136c5e6f2eb3..19f82df53e2a 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.192-tw-0.48 + 0.196 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index 3dc7d144f367..003792555c49 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 052be12dabc2..7bc59918bc6b 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index 41badf664228..c73ae23af78d 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index 2a0c71d24ee7..694f3f7b0f5f 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index c1099b5f39f1..f33edf2a5d7f 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index 2d5d38a3c467..0cab84c3d81d 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-twitter-functions diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index d93b20cc9b2e..b2170bb5413d 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 6bf3a44ae69d..00a6f5869099 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index b60e6f7eb183..261694abcdf2 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.192-tw-0.48 + 0.196 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.192-tw-0.48 + 0.196 provided From 67273de44de0a63b2bc53283136e0ca878bb973a Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 23 Mar 2018 19:08:35 -0700 Subject: [PATCH 274/331] Use total memory as peak memory bytes metrics --- .../presto/plugin/eventlistener/QueryCompletedEventScriber.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java index 477d3e302355..e6bfd714ad36 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java @@ -85,7 +85,7 @@ private static QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletedE thriftEvent.queued_time_ms = eventStat.getQueuedTime().toMillis(); thriftEvent.query_wall_time_ms = eventStat.getWallTime().toMillis(); thriftEvent.cumulative_memory_bytesecond = eventStat.getCumulativeMemory(); - thriftEvent.peak_memory_bytes = eventStat.getPeakMemoryBytes(); + thriftEvent.peak_memory_bytes = eventStat.getPeakTotalNonRevocableMemoryBytes(); thriftEvent.cpu_time_ms = eventStat.getCpuTime().toMillis(); if (eventStat.getAnalysisTime().isPresent()) { thriftEvent.analysis_time_ms = eventStat.getAnalysisTime().get().toMillis(); From 4c6c8e010b6eb02c7bba3741015430d6f174cc65 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 23 Mar 2018 19:57:22 -0700 Subject: [PATCH 275/331] fix conflict --- .../com/facebook/presto/hive/HiveCoercionRecordCursor.java | 3 --- .../src/main/java/com/facebook/presto/hive/HivePageSource.java | 2 -- .../java/com/facebook/presto/hive/AbstractTestHiveClient.java | 1 - .../main/java/com/facebook/presto/kafka/KafkaSplitManager.java | 2 +- .../facebook/presto/server/security/TestSecurityConfig.java | 2 +- 5 files changed, 2 insertions(+), 8 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveCoercionRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveCoercionRecordCursor.java index 78bf6580340d..68b8ea9b1d58 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveCoercionRecordCursor.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveCoercionRecordCursor.java @@ -28,10 +28,7 @@ import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; -import java.util.Arrays; import java.util.List; -import java.util.Objects; -import java.util.stream.Collectors; import static com.facebook.presto.hive.HiveType.HIVE_BYTE; import static com.facebook.presto.hive.HiveType.HIVE_DOUBLE; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSource.java b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSource.java index 365335c20c78..88ace837f9f7 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSource.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HivePageSource.java @@ -40,10 +40,8 @@ import java.io.IOException; import java.io.UncheckedIOException; -import java.util.Arrays; import java.util.List; import java.util.function.Function; -import java.util.stream.Collectors; import static com.facebook.presto.hive.HiveErrorCode.HIVE_CURSOR_ERROR; import static com.facebook.presto.hive.HiveType.HIVE_BYTE; 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 045718f30406..1ddb7a1ec847 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 @@ -118,7 +118,6 @@ import java.util.TimeZone; import java.util.UUID; import java.util.concurrent.ExecutorService; -import java.util.stream.Stream; import static com.facebook.presto.hive.AbstractTestHiveClient.TransactionDeleteInsertTestTag.COMMIT; import static com.facebook.presto.hive.AbstractTestHiveClient.TransactionDeleteInsertTestTag.ROLLBACK_AFTER_APPEND_PAGE; diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java index 1629efcf4f9d..056bfea7e2b2 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java @@ -64,7 +64,7 @@ public KafkaSplitManager( } @Override - public ConnectorSplitSource getSplits(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableLayoutHandle layout) + public ConnectorSplitSource getSplits(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableLayoutHandle layout, SplitSchedulingStrategy splitSchedulingStrategy) { KafkaTableHandle kafkaTableHandle = convertLayout(layout).getTable(); ZkClient zkClient = KafkaUtil.newZkClient(config.getZkEndpoint()); diff --git a/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java b/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java index 9e3c66791faf..52064739490e 100644 --- a/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java +++ b/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java @@ -37,7 +37,7 @@ public void testDefaults() public void testExplicitPropertyMappings() { Map properties = new ImmutableMap.Builder() - .put("http-server.authentication.type", "KERBEROS,PASSWORD") + .put("http-server.authentication.type", "KERBEROS,PASSWORD") .put("http-server.http.authentication.path.regex", "^/v1/statement") .build(); From 0c35ec64bb66ca997173ad6d3755d248c2b4fc5b Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 23 Mar 2018 21:37:33 -0700 Subject: [PATCH 276/331] Change version name --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial-toolkit/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory-context/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-password-authenticators/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 58 files changed, 60 insertions(+), 60 deletions(-) diff --git a/pom.xml b/pom.xml index d10923e987e2..f6439e93d43e 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.196 + 0.196-tw-0.49 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index bff7399ac8ae..a4b0d420d557 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index d004ff1dd13d..7b1f87723fc7 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 8bf490655dff..8949fbad0d2e 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 4f5fa6d563bf..cdaaf93b6089 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index a1211ba8ef4e..d53cf602b0ad 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index a008359b6794..50fdf6cbb283 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.196 + 0.196-tw-0.49 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 3ad7af423377..2de935090da1 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 09d55354885c..2ea62a29b01e 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 79593f8e48e5..7225b49fe70b 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 7c11b654c66a..b2c3312797f5 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 6fd61d4fd059..2390e74dacca 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 9a987e4b3c00..28e82f8eb187 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index f77d1888a579..b3807f898f01 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index 980220593aec..c5f727d2b1fd 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index c3ee3eaba84b..cc92231d711b 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 5394b686b579..95935c1c95b2 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 0486b9aca69a..45986141ea91 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index da80bbeaf83f..e8c0a0a22c22 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 1905b875133e..be4947bdabf0 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 640f0ad39c59..dbbbfe72c4b3 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index 825e616468b0..3ebad0746851 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index ecaf297fd20f..9d1d78f4c539 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 515c06c772a8..6b4a5540b9f2 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index 068761506a79..fadf9f267538 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.196 + 0.196-tw-0.49 presto-matching diff --git a/presto-memory-context/pom.xml b/presto-memory-context/pom.xml index 53fcee84cc2d..c09ad7f9f66b 100644 --- a/presto-memory-context/pom.xml +++ b/presto-memory-context/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-memory-context diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 7b9ce0472e59..bfc0f6bdded6 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 1aed1cfd810d..d9aef95f7900 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index f4f610700932..7f54cf4c1657 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 197d01fac3f3..c8705a60d091 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 194d92bde3df..71cbd040baab 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index c2c8cc6b4dca..b25eae864507 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-parser diff --git a/presto-password-authenticators/pom.xml b/presto-password-authenticators/pom.xml index ebf840a3c111..c5a8cb7b1a47 100644 --- a/presto-password-authenticators/pom.xml +++ b/presto-password-authenticators/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-password-authenticators diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 39826b6f3a8a..6fcb128ae8d5 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 56bac316f97e..f124f6f6347d 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 7ab38d399136..c82ab7963b4a 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.196 + 0.196-tw-0.49 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index a2fba282e2d0..50c763de1b8d 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 13b32deb7141..2efc684adb0f 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index ba7c8ccb3ab3..bf7812b94219 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 3421dbaf44bb..e4553afc4421 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index ab7dd6f956d1..124e1570799b 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 0a178b3239ae..7ee81b1524a0 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 7b59da2a926c..852cb08830a4 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index ebd88abecf0e..eb0e0a6ffce3 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index ca5e0f8ba489..4598b523abae 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 39e57a4f08e0..00ebd689dd05 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.196 + 0.196-tw-0.49 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index db25c6e73fdd..d3027a7dc7d6 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index c7298620b767..ddbcb9fad9c3 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 6f15398b9df7..8b8c91cae7c7 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.196 + 0.196-tw-0.49 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index 003792555c49..7987bcdad12b 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 7bc59918bc6b..670ad0c7d4a4 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index c73ae23af78d..b56a9d2ac497 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index 694f3f7b0f5f..55c3f5f2fabd 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index f33edf2a5d7f..70131dfb7a62 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index 0cab84c3d81d..ec79cee209b6 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-twitter-functions diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index b2170bb5413d..8ef9a1160fc9 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 00a6f5869099..591446214b77 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 261694abcdf2..d78b0cc2f35c 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196 + 0.196-tw-0.49 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.196 + 0.196-tw-0.49 provided From baa68cfc35de2d061136e334feee41bde22ad18b Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 27 Mar 2018 13:39:25 -0700 Subject: [PATCH 277/331] Fix test failure due to twitter patch in sql --- presto-hive/src/test/sql/create-test-hive13.sql | 4 ---- 1 file changed, 4 deletions(-) diff --git a/presto-hive/src/test/sql/create-test-hive13.sql b/presto-hive/src/test/sql/create-test-hive13.sql index d6e230026719..54747332d993 100644 --- a/presto-hive/src/test/sql/create-test-hive13.sql +++ b/presto-hive/src/test/sql/create-test-hive13.sql @@ -104,12 +104,10 @@ CREATE TABLE presto_test_types_parquet ( , t_struct STRUCT , t_complex MAP>> ) -PARTITIONED BY (dummy INT) STORED AS PARQUET ; INSERT INTO TABLE presto_test_types_parquet -PARTITION (dummy=0) SELECT t_string , t_varchar @@ -130,8 +128,6 @@ SELECT FROM presto_test_types_textfile ; -ALTER TABLE presto_test_types_parquet -CHANGE COLUMN t_struct t_struct STRUCT; ALTER TABLE presto_test_types_textfile ADD COLUMNS (new_column INT); ALTER TABLE presto_test_types_sequencefile ADD COLUMNS (new_column INT); From e01096247602b38f3e373183bdbab1609161f5f3 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 11 Apr 2018 17:16:26 -0700 Subject: [PATCH 278/331] Add ldap password authenticator and local file connector to twitter server (#149) --- presto-twitter-server/src/main/provisio/presto.xml | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/presto-twitter-server/src/main/provisio/presto.xml b/presto-twitter-server/src/main/provisio/presto.xml index 85dc97c16451..7a448a544048 100644 --- a/presto-twitter-server/src/main/provisio/presto.xml +++ b/presto-twitter-server/src/main/provisio/presto.xml @@ -32,6 +32,12 @@ + + + + + + @@ -61,4 +67,10 @@ + + + + + + From 5350b8b0e42e531d04cff5df3ab674adcd588ad8 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Mon, 4 Jun 2018 13:47:54 -0700 Subject: [PATCH 279/331] Roll back the version number to 0.196 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial-toolkit/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory-context/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-password-authenticators/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 58 files changed, 60 insertions(+), 60 deletions(-) diff --git a/pom.xml b/pom.xml index f6439e93d43e..d10923e987e2 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.196-tw-0.49 + 0.196 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index a4b0d420d557..bff7399ac8ae 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 7b1f87723fc7..d004ff1dd13d 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 8949fbad0d2e..8bf490655dff 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index cdaaf93b6089..4f5fa6d563bf 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index d53cf602b0ad..a1211ba8ef4e 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 50fdf6cbb283..a008359b6794 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.196-tw-0.49 + 0.196 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 2de935090da1..3ad7af423377 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 2ea62a29b01e..09d55354885c 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 7225b49fe70b..79593f8e48e5 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index b2c3312797f5..7c11b654c66a 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 2390e74dacca..6fd61d4fd059 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 28e82f8eb187..9a987e4b3c00 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index b3807f898f01..f77d1888a579 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index c5f727d2b1fd..980220593aec 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index cc92231d711b..c3ee3eaba84b 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 95935c1c95b2..5394b686b579 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 45986141ea91..0486b9aca69a 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index e8c0a0a22c22..da80bbeaf83f 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index be4947bdabf0..1905b875133e 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index dbbbfe72c4b3..640f0ad39c59 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index 3ebad0746851..825e616468b0 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 9d1d78f4c539..ecaf297fd20f 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 6b4a5540b9f2..515c06c772a8 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index fadf9f267538..068761506a79 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.196-tw-0.49 + 0.196 presto-matching diff --git a/presto-memory-context/pom.xml b/presto-memory-context/pom.xml index c09ad7f9f66b..53fcee84cc2d 100644 --- a/presto-memory-context/pom.xml +++ b/presto-memory-context/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-memory-context diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index bfc0f6bdded6..7b9ce0472e59 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index d9aef95f7900..1aed1cfd810d 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 7f54cf4c1657..f4f610700932 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index c8705a60d091..197d01fac3f3 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 71cbd040baab..194d92bde3df 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index b25eae864507..c2c8cc6b4dca 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-parser diff --git a/presto-password-authenticators/pom.xml b/presto-password-authenticators/pom.xml index c5a8cb7b1a47..ebf840a3c111 100644 --- a/presto-password-authenticators/pom.xml +++ b/presto-password-authenticators/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-password-authenticators diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 6fcb128ae8d5..39826b6f3a8a 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index f124f6f6347d..56bac316f97e 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index c82ab7963b4a..7ab38d399136 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.196-tw-0.49 + 0.196 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 50c763de1b8d..a2fba282e2d0 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 2efc684adb0f..13b32deb7141 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index bf7812b94219..ba7c8ccb3ab3 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index e4553afc4421..3421dbaf44bb 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index 124e1570799b..ab7dd6f956d1 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 7ee81b1524a0..0a178b3239ae 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 852cb08830a4..7b59da2a926c 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index eb0e0a6ffce3..ebd88abecf0e 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 4598b523abae..ca5e0f8ba489 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 00ebd689dd05..39e57a4f08e0 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.196-tw-0.49 + 0.196 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index d3027a7dc7d6..db25c6e73fdd 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index ddbcb9fad9c3..c7298620b767 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 8b8c91cae7c7..6f15398b9df7 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.196-tw-0.49 + 0.196 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index 7987bcdad12b..003792555c49 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 670ad0c7d4a4..7bc59918bc6b 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index b56a9d2ac497..c73ae23af78d 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index 55c3f5f2fabd..694f3f7b0f5f 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 70131dfb7a62..f33edf2a5d7f 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index ec79cee209b6..0cab84c3d81d 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-twitter-functions diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index 8ef9a1160fc9..b2170bb5413d 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 591446214b77..00a6f5869099 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index d78b0cc2f35c..261694abcdf2 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.196-tw-0.49 + 0.196 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.196-tw-0.49 + 0.196 provided From 489d3d240d6913b6e2b0145ce757ba84b34f3e27 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Mon, 4 Jun 2018 19:44:29 -0700 Subject: [PATCH 280/331] Update branched version number and fix conflicts --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial-toolkit/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 8 +++++++- presto-hive/pom.xml | 2 +- .../twitter/hive/ZookeeperServersetHiveCluster.java | 2 +- .../twitter/hive/thrift/ThriftHiveRecordCursor.java | 7 +++---- .../presto/twitter/hive/util/PooledTTransportFactory.java | 2 +- .../com/facebook/presto/hive/TestHiveFileFormats.java | 6 +++--- presto-jdbc/pom.xml | 8 +++++++- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 7 +------ presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- .../java/com/facebook/presto/type/RowParametricType.java | 4 +++- presto-matching/pom.xml | 2 +- presto-memory-context/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-password-authenticators/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- .../presto/decoder/thrift/ThriftFieldDecoder.java | 7 +++---- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 4 +--- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- .../presto/twitter/functions/TwitterStringFunctions.java | 3 +-- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 65 files changed, 87 insertions(+), 83 deletions(-) diff --git a/pom.xml b/pom.xml index 8d64919a1b0a..fae63f2ea0ca 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.203 + 0.203-tw-0.50 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 87d2b8bd7813..0bb24e48f0c1 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 378de0eae2f4..20d3dcf797b1 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 3cc797f4c7c1..ceb6187e1a07 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 0b958bd4a2ee..e5cdc6c89f87 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index d54ac84ff0da..37646599dfdf 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 5d6b5168c247..fbe58abf1466 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.203 + 0.203-tw-0.50 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 260b7d789a56..4150ed14db8a 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 34ef8d41d496..d05c3446dcbd 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index c5eb2039e5e4..8ac3da7b3da3 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index bf0a52108b93..105cee46b93e 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 539e594afe4c..45200e163470 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index ec41dd31d746..2ffdde511a20 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 8a78c81ca64e..ae49d87f0bdd 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index 87a4bff0c3ba..910082be9f09 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index 91f529c72ed7..ce497307e3bb 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 59fd78039d13..92df28805137 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-hive-hadoop2 @@ -89,6 +89,12 @@ com.facebook.presto presto-main test + + + io.netty + netty + + diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index c30f3d9ed2af..60b27a84b518 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-hive diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java index 351df92d93d2..c91912f0ff91 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/ZookeeperServersetHiveCluster.java @@ -54,7 +54,7 @@ public HiveMetastoreClient createMetastoreClient() for (HostAndPort metastore : metastores) { try { log.info("Connecting to metastore at: %s", metastore.toString()); - return clientFactory.create(metastore.getHostText(), metastore.getPort()); + return clientFactory.create(metastore.getHost(), metastore.getPort()); } catch (TTransportException e) { log.debug("Failed connecting to Hive metastore at: %s", metastore.toString()); diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java index 6cd1ba00443f..163f5f6af4a5 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java @@ -19,7 +19,6 @@ import com.facebook.presto.spi.RecordCursor; 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.DecimalType; import com.facebook.presto.spi.type.Decimals; import com.facebook.presto.spi.type.Type; @@ -577,7 +576,7 @@ private static Block serializeList(Type type, ThriftFieldIdResolver resolver, Bl currentBuilder = builder.beginBlockEntry(); } else { - currentBuilder = elementType.createBlockBuilder(new BlockBuilderStatus(), list.size()); + currentBuilder = elementType.createBlockBuilder(null, list.size()); } for (Object element : list) { @@ -606,7 +605,7 @@ private static Block serializeMap(Type type, ThriftFieldIdResolver resolver, Blo boolean builderSynthesized = false; if (builder == null) { builderSynthesized = true; - builder = type.createBlockBuilder(new BlockBuilderStatus(), 1); + builder = type.createBlockBuilder(null, 1); } BlockBuilder currentBuilder = builder.beginBlockEntry(); @@ -635,7 +634,7 @@ private static Block serializeStruct(Type type, ThriftFieldIdResolver resolver, boolean builderSynthesized = false; if (builder == null) { builderSynthesized = true; - builder = type.createBlockBuilder(new BlockBuilderStatus(), 1); + builder = type.createBlockBuilder(null, 1); } BlockBuilder currentBuilder = builder.beginBlockEntry(); diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java index 554435e12923..f823576d8389 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/util/PooledTTransportFactory.java @@ -86,7 +86,7 @@ public TTransport create() transport = new TSocket(host, port, timeoutMillis); } else { - SocketAddress address = InetSocketAddress.createUnresolved(socksProxy.getHostText(), + SocketAddress address = InetSocketAddress.createUnresolved(socksProxy.getHost(), socksProxy.getPort()); Socket socket = new Socket(new Proxy(Proxy.Type.SOCKS, address)); try { 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 c24167ebcdd2..fb40bee25044 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 @@ -501,9 +501,9 @@ public void testParquetThrift(int rowCount) public void testLZOThrift(int rowCount) throws Exception { - RowType nameType = new RowType(ImmutableList.of(createUnboundedVarcharType(), createUnboundedVarcharType()), Optional.empty()); - RowType phoneType = new RowType(ImmutableList.of(createUnboundedVarcharType(), createUnboundedVarcharType()), Optional.empty()); - RowType personType = new RowType(ImmutableList.of(nameType, INTEGER, createUnboundedVarcharType(), new ArrayType(phoneType)), Optional.empty()); + RowType nameType = RowType.anonymous(ImmutableList.of(createUnboundedVarcharType(), createUnboundedVarcharType())); + RowType phoneType = RowType.anonymous(ImmutableList.of(createUnboundedVarcharType(), createUnboundedVarcharType())); + RowType personType = RowType.anonymous(ImmutableList.of(nameType, INTEGER, createUnboundedVarcharType(), new ArrayType(phoneType))); List testColumns = ImmutableList.of( new TestColumn( diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 5a12f9de20ef..14d489fd0974 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-jdbc @@ -74,6 +74,12 @@ com.facebook.presto presto-main test + + + io.netty + netty + + diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 989c08a26de9..caa041ed2a94 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index edc53df7a8e8..a02f0ad9fe68 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index 13339741806b..91ec7a824e93 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-kafka07 @@ -55,11 +55,6 @@ guice - - com.google.inject.extensions - guice-multibindings - - javax.validation validation-api diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 5e4800b0fcae..54ae0984239f 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index bfd40b733e01..122256f126be 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-main diff --git a/presto-main/src/main/java/com/facebook/presto/type/RowParametricType.java b/presto-main/src/main/java/com/facebook/presto/type/RowParametricType.java index e2342178178c..492f41554245 100644 --- a/presto-main/src/main/java/com/facebook/presto/type/RowParametricType.java +++ b/presto-main/src/main/java/com/facebook/presto/type/RowParametricType.java @@ -13,6 +13,7 @@ */ package com.facebook.presto.type; +import com.facebook.presto.spi.type.NamedType; import com.facebook.presto.spi.type.ParameterKind; import com.facebook.presto.spi.type.ParametricType; import com.facebook.presto.spi.type.RowType; @@ -22,6 +23,7 @@ import com.facebook.presto.spi.type.TypeParameter; import java.util.List; +import java.util.Optional; import static com.google.common.base.Preconditions.checkArgument; import static java.util.stream.Collectors.toList; @@ -45,7 +47,7 @@ public String getName() public Type createType(TypeManager typeManager, List parameters) { if (parameters.isEmpty()) { - parameters.add(TypeParameter.of(new NamedType(UnknownType.NAME, UnknownType.UNKNOWN))); + parameters.add(TypeParameter.of(new NamedType(Optional.of(UnknownType.NAME), UnknownType.UNKNOWN))); } checkArgument( parameters.stream().allMatch(parameter -> parameter.getKind() == ParameterKind.NAMED_TYPE), diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index 55c0b1e9a5a4..a2540c284da7 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.203 + 0.203-tw-0.50 presto-matching diff --git a/presto-memory-context/pom.xml b/presto-memory-context/pom.xml index 31a664538393..44f49a9524e1 100644 --- a/presto-memory-context/pom.xml +++ b/presto-memory-context/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-memory-context diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 8ce423171949..8ba2b32d7d37 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index e2ed26b14409..cd37b33ff496 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 35f6de53af6e..f108fe80099e 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 4ac0d9575a23..160e84772612 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index caf7b344f1ce..c21424bd9316 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index a86e2222b42c..27363afee38c 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-parser diff --git a/presto-password-authenticators/pom.xml b/presto-password-authenticators/pom.xml index bed4c7871d41..db1f72b87227 100644 --- a/presto-password-authenticators/pom.xml +++ b/presto-password-authenticators/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-password-authenticators diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index a8ecabe333ac..ce548f66396b 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 63efe4b9fcf6..fbe314fcc05b 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index daec15262e90..050f14810548 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.203 + 0.203-tw-0.50 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 8fd17ddb8d4d..ae7d9859a3a7 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 2c81b8ba01b3..3a567eef7c1e 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index d6ae4f7e1b0b..5f66bcd807f0 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-record-decoder diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftFieldDecoder.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftFieldDecoder.java index c8be21f7c736..77e1279a8d62 100644 --- a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftFieldDecoder.java +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/thrift/ThriftFieldDecoder.java @@ -19,7 +19,6 @@ import com.facebook.presto.decoder.FieldValueProvider; 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.StandardTypes; import com.facebook.presto.spi.type.Type; import com.google.common.collect.ImmutableSet; @@ -228,7 +227,7 @@ private static Block serializeList(Type type, BlockBuilder builder, Object objec currentBuilder = builder.beginBlockEntry(); } else { - currentBuilder = elementType.createBlockBuilder(new BlockBuilderStatus(), list.size()); + currentBuilder = elementType.createBlockBuilder(null, list.size()); } for (Object element : list) { @@ -263,7 +262,7 @@ private static Block serializeMap(Type type, BlockBuilder builder, Object object currentBuilder = builder.beginBlockEntry(); } else { - currentBuilder = type.createBlockBuilder(new BlockBuilderStatus(), map.size()); + currentBuilder = type.createBlockBuilder(null, map.size()); } for (Map.Entry entry : map.entrySet()) { @@ -298,7 +297,7 @@ private static Block serializeStruct(Type type, BlockBuilder builder, Object obj currentBuilder = builder.beginBlockEntry(); } else { - currentBuilder = type.createBlockBuilder(new BlockBuilderStatus(), typeParameters.size()); + currentBuilder = type.createBlockBuilder(null, typeParameters.size()); } for (int i = 0; i < typeParameters.size(); i++) { diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index d227b1bb41f8..0ea4023a886a 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index c06f9727c15a..0bef5ce03bb0 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index c6efca1acab3..7256f5bae4e1 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 0969ab4aae65..5934c1061b89 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index b4ff169ed3e1..4f3d52b59e13 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 7d5a257a68ab..abb3e549d2c2 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index cabc90930c1c..f44215d90365 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.203 + 0.203-tw-0.50 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 2955728fcff8..062f6537c523 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 7ee55814bdc5..25808a6bb137 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index af58047637c6..86cc42e57aec 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.203 + 0.203-tw-0.50 presto-tests @@ -125,8 +125,6 @@ com.google.guava guava - - 21.0 diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index 9baae2a32ff0..d7d41c1d8afe 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 4684a5edddc1..e922869522e0 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index 2769add60d29..f7721f792734 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index e934b34c8298..d095c668a5ce 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 88d4fa686b0f..f96aed06ed31 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index 6ef704e8ab28..0802772c865c 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-twitter-functions diff --git a/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterStringFunctions.java b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterStringFunctions.java index dac9cf846a9a..9f43baaafe3a 100644 --- a/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterStringFunctions.java +++ b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterStringFunctions.java @@ -17,7 +17,6 @@ import com.facebook.presto.spi.PrestoException; 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.function.Description; import com.facebook.presto.spi.function.LiteralParameters; import com.facebook.presto.spi.function.ScalarFunction; @@ -65,7 +64,7 @@ public static Block str2array(@SqlType("varchar(x)") Slice utf8, @SqlType(Standa checkCondition(limit <= Integer.MAX_VALUE, INVALID_FUNCTION_ARGUMENT, "Limit is too large"); checkCondition(length > 0, INVALID_FUNCTION_ARGUMENT, "Length must be positive"); checkCondition(length <= Integer.MAX_VALUE, INVALID_FUNCTION_ARGUMENT, "Length is too large"); - BlockBuilder parts = VARCHAR.createBlockBuilder(new BlockBuilderStatus(), 1, Ints.saturatedCast(length)); + BlockBuilder parts = VARCHAR.createBlockBuilder(null, 1, Ints.saturatedCast(length)); // If limit is one, the last and only element is the complete string if (limit == 1) { VARCHAR.writeSlice(parts, utf8); diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index 48eda2f04c58..7ead242e55f8 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 07d801e8cd8b..ed49787fd4fe 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index c787fa74bde3..6e8f229cd377 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.203-tw-0.50 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.203 + 0.203-tw-0.50 provided From c71d5a98de07cf5581085d1df408a8930e94d1f8 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 7 Jun 2018 21:36:00 -0700 Subject: [PATCH 281/331] Support case-insensitive parquet columns look up --- .../hive/parquet/ParquetPageSource.java | 3 +- .../presto/hive/parquet/ParquetTypeUtils.java | 32 +++++++++++++++++++ .../java/parquet/io/ColumnIOConverter.java | 3 +- 3 files changed, 36 insertions(+), 2 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSource.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSource.java index d69a11910ab7..248f86f88ecb 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSource.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSource.java @@ -38,6 +38,7 @@ import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveErrorCode.HIVE_BAD_DATA; import static com.facebook.presto.hive.HiveErrorCode.HIVE_CURSOR_ERROR; +import static com.facebook.presto.hive.parquet.ParquetTypeUtils.findColumnIObyName; import static com.facebook.presto.hive.parquet.ParquetTypeUtils.findFieldIndexByName; import static com.facebook.presto.hive.parquet.ParquetTypeUtils.getParquetType; import static com.google.common.base.Preconditions.checkState; @@ -105,7 +106,7 @@ public ParquetPageSource( } else { String columnName = useParquetColumnNames ? name : fileSchema.getFields().get(column.getHiveColumnIndex()).getName(); - fieldsBuilder.add(constructField(type, messageColumnIO.getChild(columnName))); + fieldsBuilder.add(constructField(type, findColumnIObyName(messageColumnIO, columnName))); } } types = typesBuilder.build(); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java index 7c27abba7754..1f704f478ed2 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java @@ -234,6 +234,19 @@ public static int findFieldIndexByName(MessageType fileSchema, String name) return fieldIndex; } + // Find the ColumnIO by name following the same logic as findParquetTypeByName + public static ColumnIO findColumnIObyName(GroupColumnIO groupColumnIO, String name) + { + // direct match and case-insensitive match + ColumnIO columnIO = getColumnIOByName(groupColumnIO, name); + + if (columnIO == null && name.endsWith("_")) { + return findColumnIObyName(groupColumnIO, name.substring(0, name.length() - 1)); + } + + return columnIO; + } + public static ParquetEncoding getParquetEncoding(Encoding encoding) { switch (encoding) { @@ -274,6 +287,25 @@ private static parquet.schema.Type getParquetTypeByName(String columnName, Messa return null; } + private static ColumnIO getColumnIOByName(GroupColumnIO groupColumnIO, String name) + { + ColumnIO columnIO = groupColumnIO.getChild(name); + + if (columnIO != null) { + return columnIO; + } + + // parquet is case-sensitive, but hive is not. all hive columns get converted to lowercase + // check for direct match above but if no match found, try case-insensitive match + for (int i = 0; i < groupColumnIO.getChildrenCount(); i++) { + if (groupColumnIO.getChild(i).getName().equalsIgnoreCase(name)) { + return groupColumnIO.getChild(i); + } + } + + return null; + } + public static Optional createDecimalType(RichColumnDescriptor descriptor) { if (descriptor.getPrimitiveType().getOriginalType() != DECIMAL) { diff --git a/presto-hive/src/main/java/parquet/io/ColumnIOConverter.java b/presto-hive/src/main/java/parquet/io/ColumnIOConverter.java index 1a7e34fd0bfa..060227f9da71 100644 --- a/presto-hive/src/main/java/parquet/io/ColumnIOConverter.java +++ b/presto-hive/src/main/java/parquet/io/ColumnIOConverter.java @@ -27,6 +27,7 @@ import java.util.Locale; import java.util.Optional; +import static com.facebook.presto.hive.parquet.ParquetTypeUtils.findColumnIObyName; import static com.facebook.presto.hive.parquet.ParquetTypeUtils.getArrayElementColumn; import static com.facebook.presto.hive.parquet.ParquetTypeUtils.getMapKeyValueColumn; import static com.facebook.presto.spi.type.StandardTypes.ARRAY; @@ -60,7 +61,7 @@ public static Optional constructField(Type type, ColumnIO columnIO) for (int i = 0; i < fields.size(); i++) { NamedTypeSignature namedTypeSignature = fields.get(i).getNamedTypeSignature(); String name = namedTypeSignature.getName().get().toLowerCase(Locale.ENGLISH); - Optional field = constructField(parameters.get(i), groupColumnIO.getChild(name)); + Optional field = constructField(parameters.get(i), findColumnIObyName(groupColumnIO, name)); structHasParameters |= field.isPresent(); fileldsBuilder.add(field); } From fdfce171f31020d329db39d9c41685065bdef2fd Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 6 Jun 2018 14:28:28 -0700 Subject: [PATCH 282/331] Fix kill url --- presto-main/src/main/resources/webapp/assets/query.js | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-main/src/main/resources/webapp/assets/query.js b/presto-main/src/main/resources/webapp/assets/query.js index 53eb5df421d3..ba6b68bfc894 100644 --- a/presto-main/src/main/resources/webapp/assets/query.js +++ b/presto-main/src/main/resources/webapp/assets/query.js @@ -892,7 +892,7 @@ let QueryDetail = React.createClass({

From 67b7cfa227fd7caac77ee04bb843f583853855d5 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 7 Jun 2018 13:21:26 -0700 Subject: [PATCH 283/331] Add --user option in cli --- .../src/main/java/com/facebook/presto/cli/ClientOptions.java | 1 + 1 file changed, 1 insertion(+) 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 de1a75ca35ba..8a755148ab86 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 @@ -77,6 +77,7 @@ public class ClientOptions // Pick the user name for the logged in user. // Do not let it be overridden by users. + @Option(name = "--user", title = "user", description = "Username") public String user = new UnixSystem().getUsername(); @Option(name = "--truststore-path", title = "truststore path", description = "Truststore path") From 3fd16571f72ec106358996f99b316804caf7ad36 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 14 Jun 2018 16:07:22 -0700 Subject: [PATCH 284/331] Update the key for the memory reservation bytes (#154) --- .../twitter/presto/plugin/eventlistener/QueryStatsHelper.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java index e57cedda3ff0..82dafc9c8b40 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java @@ -135,7 +135,7 @@ private static OperatorStats getOperatorStat(JsonObject obj) operatorStats.finish_wall_millis = getMillisOrNegativeOne(obj.getString("finishWall")); operatorStats.finish_cpu_millis = getMillisOrNegativeOne(obj.getString("finishCpu")); operatorStats.finish_user_millis = getMillisOrNegativeOne(obj.getString("finishUser")); - operatorStats.memory_reservation_bytes = getBytesOrNegativeOne(obj.getString("memoryReservation")); + operatorStats.memory_reservation_bytes = getBytesOrNegativeOne(obj.getString("userMemoryReservation")); operatorStats.system_memory_reservation_bytes = getBytesOrNegativeOne(obj.getString("systemMemoryReservation")); } catch (Exception e) { From 0126bf1aab0e2f9f929689fcb88fbe6f11c0cfed Mon Sep 17 00:00:00 2001 From: Hao Luo Date: Tue, 19 Jun 2018 22:24:09 -0700 Subject: [PATCH 285/331] Fix backward-compatibility processing for array with one group field (#155) According to Parquet spec: If the repeated field is a group with one field and is named either array or uses the LIST-annotated group's name with _tuple appended then the repeated type is the element type and elements are required. https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#lists (cherry picked from commit 346f38e) --- .../presto/hive/parquet/ParquetTypeUtils.java | 11 +++- .../java/parquet/io/ColumnIOConverter.java | 10 ++-- .../parquet/AbstractTestParquetReader.java | 44 +++++++++++++-- .../presto/hive/parquet/ParquetTester.java | 53 +++++++++++++------ .../SingleLevelArraySchemaConverter.java | 2 +- .../write/TestDataWritableWriteSupport.java | 8 ++- .../parquet/write/TestDataWritableWriter.java | 21 ++++---- .../write/TestMapredParquetOutputFormat.java | 4 +- 8 files changed, 113 insertions(+), 40 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java index 1f704f478ed2..ad03201063ea 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java @@ -71,6 +71,13 @@ public static GroupColumnIO getMapKeyValueColumn(GroupColumnIO groupColumnIO) return groupColumnIO; } + /* For backward-compatibility, the type of elements in LIST-annotated structures should always be determined by the following rules: + * 1. If the repeated field is not a group, then its type is the element type and elements are required. + * 2. If the repeated field is a group with multiple fields, then its type is the element type and elements are required. + * 3. If the repeated field is a group with one field and is named either array or uses the LIST-annotated group's name with _tuple appended then the repeated type is the element type and elements are required. + * 4. Otherwise, the repeated field's type is the element type with the repeated field's repetition. + * https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#lists + */ public static ColumnIO getArrayElementColumn(ColumnIO columnIO) { while (columnIO instanceof GroupColumnIO && !columnIO.getType().isRepetition(REPEATED)) { @@ -86,7 +93,9 @@ public static ColumnIO getArrayElementColumn(ColumnIO columnIO) */ if (columnIO instanceof GroupColumnIO && columnIO.getType().getOriginalType() == null && - ((GroupColumnIO) columnIO).getChildrenCount() == 1) { + ((GroupColumnIO) columnIO).getChildrenCount() == 1 && + !columnIO.getName().equals("array") && + !columnIO.getName().equals(columnIO.getParent().getName() + "_tuple")) { return ((GroupColumnIO) columnIO).getChild(0); } diff --git a/presto-hive/src/main/java/parquet/io/ColumnIOConverter.java b/presto-hive/src/main/java/parquet/io/ColumnIOConverter.java index 060227f9da71..e657878e54f3 100644 --- a/presto-hive/src/main/java/parquet/io/ColumnIOConverter.java +++ b/presto-hive/src/main/java/parquet/io/ColumnIOConverter.java @@ -55,7 +55,7 @@ public static Optional constructField(Type type, ColumnIO columnIO) if (ROW.equals(type.getTypeSignature().getBase())) { GroupColumnIO groupColumnIO = (GroupColumnIO) columnIO; List parameters = type.getTypeParameters(); - ImmutableList.Builder> fileldsBuilder = ImmutableList.builder(); + ImmutableList.Builder> fieldsBuilder = ImmutableList.builder(); List fields = type.getTypeSignature().getParameters(); boolean structHasParameters = false; for (int i = 0; i < fields.size(); i++) { @@ -63,10 +63,10 @@ public static Optional constructField(Type type, ColumnIO columnIO) String name = namedTypeSignature.getName().get().toLowerCase(Locale.ENGLISH); Optional field = constructField(parameters.get(i), findColumnIObyName(groupColumnIO, name)); structHasParameters |= field.isPresent(); - fileldsBuilder.add(field); + fieldsBuilder.add(field); } if (structHasParameters) { - return Optional.of(new GroupField(type, repetitionLevel, definitionLevel, required, fileldsBuilder.build())); + return Optional.of(new GroupField(type, repetitionLevel, definitionLevel, required, fieldsBuilder.build())); } return Optional.empty(); } @@ -74,7 +74,7 @@ else if (MAP.equals(type.getTypeSignature().getBase())) { GroupColumnIO groupColumnIO = (GroupColumnIO) columnIO; MapType mapType = (MapType) type; GroupColumnIO keyValueColumnIO = getMapKeyValueColumn(groupColumnIO); - if (keyValueColumnIO.getChildrenCount() < 2) { + if (keyValueColumnIO.getChildrenCount() != 2) { return Optional.empty(); } Optional keyField = constructField(mapType.getKeyType(), keyValueColumnIO.getChild(0)); @@ -84,7 +84,7 @@ else if (MAP.equals(type.getTypeSignature().getBase())) { else if (ARRAY.equals(type.getTypeSignature().getBase())) { GroupColumnIO groupColumnIO = (GroupColumnIO) columnIO; List types = type.getTypeParameters(); - if (groupColumnIO.getChildrenCount() == 0) { + if (groupColumnIO.getChildrenCount() != 1) { return Optional.empty(); } Optional field = constructField(types.get(0), getArrayElementColumn(groupColumnIO.getChild(0))); diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/parquet/AbstractTestParquetReader.java b/presto-hive/src/test/java/com/facebook/presto/hive/parquet/AbstractTestParquetReader.java index 909816901c55..8039f475a390 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/parquet/AbstractTestParquetReader.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/parquet/AbstractTestParquetReader.java @@ -206,7 +206,7 @@ public void testCustomSchemaArrayOfStucts() Iterable> values = createTestArrays(structs); List structFieldNames = asList("a", "b", "c"); Type structType = RowType.from(asList(field("a", BIGINT), field("b", BOOLEAN), field("c", VARCHAR))); - tester.testRoundTrip( + tester.testSingleLevelArrayRoundTrip( getStandardListObjectInspector(getStandardStructObjectInspector(structFieldNames, asList(javaLongObjectInspector, javaBooleanObjectInspector, javaStringObjectInspector))), values, values, new ArrayType(structType), Optional.of(customSchemaArrayOfStucts)); } @@ -379,6 +379,44 @@ public void testSingleLevelArrayOfMapOfStruct() values, values, new ArrayType(mapType(INTEGER, structType))); } + @Test + public void testSingleLevelArrayOfStructOfSingleElement() + throws Exception + { + Iterable structs = createTestStructs(transform(intsBetween(0, 31_234), Object::toString)); + Iterable> values = createTestArrays(structs); + List structFieldNames = singletonList("test"); + Type structType = RowType.from(singletonList(field("test", VARCHAR))); + tester.testRoundTrip( + getStandardListObjectInspector(getStandardStructObjectInspector(structFieldNames, singletonList(javaStringObjectInspector))), + values, values, new ArrayType(structType)); + tester.testSingleLevelArraySchemaRoundTrip( + getStandardListObjectInspector(getStandardStructObjectInspector(structFieldNames, singletonList(javaStringObjectInspector))), + values, values, new ArrayType(structType)); + } + + @Test + public void testSingleLevelArrayOfStructOfStructOfSingleElement() + throws Exception + { + Iterable structs = createTestStructs(transform(intsBetween(0, 31_234), Object::toString)); + Iterable structsOfStructs = createTestStructs(structs); + Iterable> values = createTestArrays(structsOfStructs); + List structFieldNames = singletonList("test"); + List structsOfStructsFieldNames = singletonList("test"); + Type structType = RowType.from(singletonList(field("test", VARCHAR))); + Type structsOfStructsType = RowType.from(singletonList(field("test", structType))); + ObjectInspector structObjectInspector = getStandardStructObjectInspector(structFieldNames, singletonList(javaStringObjectInspector)); + tester.testRoundTrip( + getStandardListObjectInspector( + getStandardStructObjectInspector(structsOfStructsFieldNames, singletonList(structObjectInspector))), + values, values, new ArrayType(structsOfStructsType)); + tester.testSingleLevelArraySchemaRoundTrip( + getStandardListObjectInspector( + getStandardStructObjectInspector(structsOfStructsFieldNames, singletonList(structObjectInspector))), + values, values, new ArrayType(structsOfStructsType)); + } + @Test public void testArrayOfMapOfArray() throws Exception @@ -1085,7 +1123,7 @@ public void testSchemaWithRequiredOptionalRequired2Fields() ObjectInspector eInspector = getStandardStructObjectInspector(singletonList("f"), singletonList(fInspector)); tester.testRoundTrip(asList(aInspector, eInspector), new Iterable[] {aValues, eValues}, new Iterable[] {aValues, eValues}, - asList("a", "e"), asList(aType, eType), Optional.of(parquetSchema)); + asList("a", "e"), asList(aType, eType), Optional.of(parquetSchema), false); } @Test @@ -1098,7 +1136,7 @@ public void testOldAvroArray() " }" + "} "); Iterable> nonNullArrayElements = createTestArrays(intsBetween(0, 31_234)); - tester.testRoundTrip(getStandardListObjectInspector(javaIntObjectInspector), nonNullArrayElements, nonNullArrayElements, new ArrayType(INTEGER), Optional.of(parquetMrAvroSchema)); + tester.testSingleLevelArrayRoundTrip(getStandardListObjectInspector(javaIntObjectInspector), nonNullArrayElements, nonNullArrayElements, new ArrayType(INTEGER), Optional.of(parquetMrAvroSchema)); } @Test 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 172a83fe91f3..e48457fa59bb 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 @@ -152,10 +152,10 @@ public void testSingleLevelArraySchemaRoundTrip(ObjectInspector objectInspector, { ArrayList typeInfos = TypeInfoUtils.getTypeInfosFromTypeString(objectInspector.getTypeName()); MessageType schema = SingleLevelArraySchemaConverter.convert(TEST_COLUMN, typeInfos); - testRoundTrip(objectInspector, writeValues, readValues, type, Optional.of(schema)); + testSingleLevelArrayRoundTrip(objectInspector, writeValues, readValues, type, Optional.of(schema)); if (objectInspector.getTypeName().contains("map<")) { schema = SingleLevelArrayMapKeyValuesSchemaConverter.convert(TEST_COLUMN, typeInfos); - testRoundTrip(objectInspector, writeValues, readValues, type, Optional.of(schema)); + testSingleLevelArrayRoundTrip(objectInspector, writeValues, readValues, type, Optional.of(schema)); } } @@ -163,7 +163,8 @@ public void testRoundTrip(ObjectInspector objectInspector, Iterable writeValu throws Exception { // just the values - testRoundTripType(singletonList(objectInspector), new Iterable[] {writeValues}, new Iterable[] {readValues}, TEST_COLUMN, singletonList(type), Optional.empty()); + testRoundTripType(singletonList(objectInspector), new Iterable[] {writeValues}, + new Iterable[] {readValues}, TEST_COLUMN, singletonList(type), Optional.empty(), false); // all nulls assertRoundTrip(singletonList(objectInspector), new Iterable[] {transform(writeValues, constant(null))}, @@ -173,7 +174,7 @@ public void testRoundTrip(ObjectInspector objectInspector, Iterable writeValu MessageType schema = MapKeyValuesSchemaConverter.convert(TEST_COLUMN, typeInfos); // just the values testRoundTripType(singletonList(objectInspector), new Iterable[] {writeValues}, new Iterable[] { - readValues}, TEST_COLUMN, singletonList(type), Optional.of(schema)); + readValues}, TEST_COLUMN, singletonList(type), Optional.of(schema), false); // all nulls assertRoundTrip(singletonList(objectInspector), new Iterable[] {transform(writeValues, constant(null))}, @@ -184,34 +185,40 @@ public void testRoundTrip(ObjectInspector objectInspector, Iterable writeValu public void testRoundTrip(ObjectInspector objectInspector, Iterable writeValues, Iterable readValues, Type type, Optional parquetSchema) throws Exception { - testRoundTrip(singletonList(objectInspector), new Iterable[] {writeValues}, new Iterable[] {readValues}, TEST_COLUMN, singletonList(type), parquetSchema); + testRoundTrip(singletonList(objectInspector), new Iterable[] {writeValues}, new Iterable[] {readValues}, TEST_COLUMN, singletonList(type), parquetSchema, false); } - public void testRoundTrip(List objectInspectors, Iterable[] writeValues, Iterable[] readValues, List columnNames, List columnTypes, Optional parquetSchema) + public void testSingleLevelArrayRoundTrip(ObjectInspector objectInspector, Iterable writeValues, Iterable readValues, Type type, Optional parquetSchema) + throws Exception + { + testRoundTrip(singletonList(objectInspector), new Iterable[] {writeValues}, new Iterable[] {readValues}, TEST_COLUMN, singletonList(type), parquetSchema, true); + } + + public void testRoundTrip(List objectInspectors, Iterable[] writeValues, Iterable[] readValues, List columnNames, List columnTypes, Optional parquetSchema, boolean singleLevelArray) throws Exception { // just the values - testRoundTripType(objectInspectors, writeValues, readValues, columnNames, columnTypes, parquetSchema); + testRoundTripType(objectInspectors, writeValues, readValues, columnNames, columnTypes, parquetSchema, singleLevelArray); // all nulls - assertRoundTrip(objectInspectors, transformToNulls(writeValues), transformToNulls(readValues), columnNames, columnTypes, parquetSchema); + assertRoundTrip(objectInspectors, transformToNulls(writeValues), transformToNulls(readValues), columnNames, columnTypes, parquetSchema, singleLevelArray); } private void testRoundTripType(List objectInspectors, Iterable[] writeValues, Iterable[] readValues, - List columnNames, List columnTypes, Optional parquetSchema) + List columnNames, List columnTypes, Optional parquetSchema, boolean singleLevelArray) throws Exception { // forward order - assertRoundTrip(objectInspectors, writeValues, readValues, columnNames, columnTypes, parquetSchema); + assertRoundTrip(objectInspectors, writeValues, readValues, columnNames, columnTypes, parquetSchema, singleLevelArray); // reverse order - assertRoundTrip(objectInspectors, reverse(writeValues), reverse(readValues), columnNames, columnTypes, parquetSchema); + assertRoundTrip(objectInspectors, reverse(writeValues), reverse(readValues), columnNames, columnTypes, parquetSchema, singleLevelArray); // forward order with nulls - assertRoundTrip(objectInspectors, insertNullEvery(5, writeValues), insertNullEvery(5, readValues), columnNames, columnTypes, parquetSchema); + assertRoundTrip(objectInspectors, insertNullEvery(5, writeValues), insertNullEvery(5, readValues), columnNames, columnTypes, parquetSchema, singleLevelArray); // reverse order with nulls - assertRoundTrip(objectInspectors, insertNullEvery(5, reverse(writeValues)), insertNullEvery(5, reverse(readValues)), columnNames, columnTypes, parquetSchema); + assertRoundTrip(objectInspectors, insertNullEvery(5, reverse(writeValues)), insertNullEvery(5, reverse(readValues)), columnNames, columnTypes, parquetSchema, singleLevelArray); } void assertRoundTrip(List objectInspectors, @@ -221,6 +228,18 @@ void assertRoundTrip(List objectInspectors, List columnTypes, Optional parquetSchema) throws Exception + { + assertRoundTrip(objectInspectors, writeValues, readValues, columnNames, columnTypes, parquetSchema, false); + } + + void assertRoundTrip(List objectInspectors, + Iterable[] writeValues, + Iterable[] readValues, + List columnNames, + List columnTypes, + Optional parquetSchema, + boolean singleLevelArray) + throws Exception { for (WriterVersion version : versions) { for (CompressionCodecName compressionCodecName : compressions) { @@ -236,7 +255,8 @@ void assertRoundTrip(List objectInspectors, createTableProperties(columnNames, objectInspectors), getStandardStructObjectInspector(columnNames, objectInspectors), getIterators(writeValues), - parquetSchema); + parquetSchema, + singleLevelArray); assertFileContents( tempFile.getFile(), getIterators(readValues), @@ -383,10 +403,11 @@ private static DataSize writeParquetColumn(JobConf jobConf, Properties tableProperties, SettableStructObjectInspector objectInspector, Iterator[] valuesByField, - Optional parquetSchema) + Optional parquetSchema, + boolean singleLevelArray) throws Exception { - RecordWriter recordWriter = new TestMapredParquetOutputFormat(parquetSchema) + RecordWriter recordWriter = new TestMapredParquetOutputFormat(parquetSchema, singleLevelArray) .getHiveRecordWriter( jobConf, new Path(outputFile.toURI()), diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/parquet/write/SingleLevelArraySchemaConverter.java b/presto-hive/src/test/java/com/facebook/presto/hive/parquet/write/SingleLevelArraySchemaConverter.java index 3f6514a1923c..946520190eaa 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/parquet/write/SingleLevelArraySchemaConverter.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/parquet/write/SingleLevelArraySchemaConverter.java @@ -162,7 +162,7 @@ else if (typeInfo.getCategory().equals(Category.UNION)) { private static GroupType convertArrayType(final String name, final ListTypeInfo typeInfo, final Repetition repetition) { final TypeInfo subType = typeInfo.getListElementTypeInfo(); - return listWrapper(name, OriginalType.LIST, convertType("array_element", subType, Repetition.REPEATED), repetition); + return listWrapper(name, OriginalType.LIST, convertType("array", subType, Repetition.REPEATED), repetition); } // An optional group containing multiple elements diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/parquet/write/TestDataWritableWriteSupport.java b/presto-hive/src/test/java/com/facebook/presto/hive/parquet/write/TestDataWritableWriteSupport.java index 0e605ca7eed0..85b980009c3f 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/parquet/write/TestDataWritableWriteSupport.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/parquet/write/TestDataWritableWriteSupport.java @@ -31,6 +31,12 @@ class TestDataWritableWriteSupport { private TestDataWritableWriter writer; private MessageType schema; + private boolean singleLevelArray; + + public TestDataWritableWriteSupport(boolean singleLevelArray) + { + this.singleLevelArray = singleLevelArray; + } @Override public WriteContext init(final Configuration configuration) @@ -42,7 +48,7 @@ public WriteContext init(final Configuration configuration) @Override public void prepareForWrite(final RecordConsumer recordConsumer) { - writer = new TestDataWritableWriter(recordConsumer, schema); + writer = new TestDataWritableWriter(recordConsumer, schema, singleLevelArray); } @Override diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/parquet/write/TestDataWritableWriter.java b/presto-hive/src/test/java/com/facebook/presto/hive/parquet/write/TestDataWritableWriter.java index cae5db605954..c9ffd88f26de 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/parquet/write/TestDataWritableWriter.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/parquet/write/TestDataWritableWriter.java @@ -54,7 +54,7 @@ /** * This class is copied from org.apache.hadoop.hive.ql.io.parquet.write.DataWritableWriter * and extended to support empty arrays and maps (HIVE-13632). - * Additionally, there is a support for arrays without include an inner element layer and + * Additionally, there is a support for arrays without an inner element layer and * support for maps where MAP_KEY_VALUE is incorrectly used in place of MAP * for backward-compatibility rules testing (https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#lists) */ @@ -63,11 +63,13 @@ public class TestDataWritableWriter private static final Logger log = Logger.get(DataWritableWriter.class); private final RecordConsumer recordConsumer; private final GroupType schema; + private final boolean singleLevelArray; - public TestDataWritableWriter(final RecordConsumer recordConsumer, final GroupType schema) + public TestDataWritableWriter(final RecordConsumer recordConsumer, final GroupType schema, boolean singleLevelArray) { this.recordConsumer = recordConsumer; this.schema = schema; + this.singleLevelArray = singleLevelArray; } /** @@ -139,7 +141,12 @@ private void writeValue(final Object value, final ObjectInspector inspector, fin if (originalType != null && originalType.equals(OriginalType.LIST)) { checkInspectorCategory(inspector, ObjectInspector.Category.LIST); - writeArray(value, (ListObjectInspector) inspector, groupType); + if (singleLevelArray) { + writeSingleLevelArray(value, (ListObjectInspector) inspector, groupType); + } + else { + writeArray(value, (ListObjectInspector) inspector, groupType); + } } else if (originalType != null && (originalType.equals(OriginalType.MAP) || originalType.equals(OriginalType.MAP_KEY_VALUE))) { checkInspectorCategory(inspector, ObjectInspector.Category.MAP); @@ -198,16 +205,8 @@ private void writeGroup(final Object value, final StructObjectInspector inspecto */ private void writeArray(final Object value, final ListObjectInspector inspector, final GroupType type) { - if (type.getType(0).isPrimitive()) { - writeSingleLevelArray(value, inspector, type); - return; - } // Get the internal array structure GroupType repeatedType = type.getType(0).asGroupType(); - if (repeatedType.getOriginalType() != null || repeatedType.getFieldCount() > 1) { - writeSingleLevelArray(value, inspector, type); - return; - } recordConsumer.startGroup(); List arrayValues = inspector.getList(value); diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/parquet/write/TestMapredParquetOutputFormat.java b/presto-hive/src/test/java/com/facebook/presto/hive/parquet/write/TestMapredParquetOutputFormat.java index db58de6bed07..6757a702ee46 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/parquet/write/TestMapredParquetOutputFormat.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/parquet/write/TestMapredParquetOutputFormat.java @@ -41,9 +41,9 @@ public class TestMapredParquetOutputFormat { private final Optional schema; - public TestMapredParquetOutputFormat(Optional schema) + public TestMapredParquetOutputFormat(Optional schema, boolean singleLevelArray) { - super(new ParquetOutputFormat<>(new TestDataWritableWriteSupport())); + super(new ParquetOutputFormat<>(new TestDataWritableWriteSupport(singleLevelArray))); this.schema = requireNonNull(schema, "schema is null"); } From 216649032a0e16ca409605ce9cb88c7403ebac56 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 27 Jun 2018 01:15:48 -0700 Subject: [PATCH 286/331] Set default soft memory limit to Long.MAX_VALUE (#156) This fixes #10383, where deployments using the legacy resource group configuration manager were unable to run more than a single query that used memory concurrently. --- .../presto/execution/resourceGroups/InternalResourceGroup.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-main/src/main/java/com/facebook/presto/execution/resourceGroups/InternalResourceGroup.java b/presto-main/src/main/java/com/facebook/presto/execution/resourceGroups/InternalResourceGroup.java index a7def1d54cf3..5f3a92a07ce5 100644 --- a/presto-main/src/main/java/com/facebook/presto/execution/resourceGroups/InternalResourceGroup.java +++ b/presto-main/src/main/java/com/facebook/presto/execution/resourceGroups/InternalResourceGroup.java @@ -86,7 +86,7 @@ public class InternalResourceGroup // Configuration // ============= @GuardedBy("root") - private long softMemoryLimitBytes; + private long softMemoryLimitBytes = Long.MAX_VALUE; @GuardedBy("root") private int softConcurrencyLimit; @GuardedBy("root") From 54a77d16428697ce60ca9d5e6deaa947b2f85127 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 27 Jun 2018 02:15:44 -0700 Subject: [PATCH 287/331] Update the key in the StageStats (#158) --- .../twitter/presto/plugin/eventlistener/QueryStatsHelper.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java index 82dafc9c8b40..b38c5c90e9f3 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java @@ -79,8 +79,8 @@ private static QueryStageInfo getQueryStageInfo(int stageId, JsonObject stage) stageInfo.output_data_size_bytes = getBytesOrNegativeOne(stageStats.getString("outputDataSize")); stageInfo.completed_tasks = stageStats.getInt("completedTasks"); stageInfo.completed_drivers = stageStats.getInt("completedDrivers"); - stageInfo.cumulative_memory = stageStats.getJsonNumber("cumulativeMemory").doubleValue(); - stageInfo.peak_memory_reservation_bytes = getBytesOrNegativeOne(stageStats.getString("peakMemoryReservation")); + stageInfo.cumulative_memory = stageStats.getJsonNumber("cumulativeUserMemory").doubleValue(); + stageInfo.peak_memory_reservation_bytes = getBytesOrNegativeOne(stageStats.getString("peakUserMemoryReservation")); stageInfo.total_scheduled_time_millis = getMillisOrNegativeOne(stageStats.getString("totalScheduledTime")); stageInfo.total_cpu_time_millis = getMillisOrNegativeOne(stageStats.getString("totalCpuTime")); stageInfo.total_user_time_millis = getMillisOrNegativeOne(stageStats.getString("totalUserTime")); From c2428ab2b44470852952755c3b58308037df87b6 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 27 Jun 2018 02:48:08 -0700 Subject: [PATCH 288/331] Release 0.203-tw-0.51 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial-toolkit/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory-context/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-password-authenticators/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 58 files changed, 60 insertions(+), 60 deletions(-) diff --git a/pom.xml b/pom.xml index fae63f2ea0ca..fd6a372540c6 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.203-tw-0.50 + 0.203-tw-0.51 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 0bb24e48f0c1..0e9e73675825 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 20d3dcf797b1..02be0b4a6ad9 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index ceb6187e1a07..251ce499962c 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index e5cdc6c89f87..0369c817e12d 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 37646599dfdf..b53870e0c222 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index fbe58abf1466..b7105da7bbc1 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.203-tw-0.50 + 0.203-tw-0.51 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 4150ed14db8a..488bcbf2fc0b 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index d05c3446dcbd..38244a3ceeea 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 8ac3da7b3da3..2615db84983b 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 105cee46b93e..7dcbebe1475b 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 45200e163470..84409e86cd05 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 2ffdde511a20..57e3ee9243b1 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index ae49d87f0bdd..be9928d37bbe 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index 910082be9f09..7ccca0a2dfd9 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index ce497307e3bb..c4390b18057e 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 92df28805137..35303f4dd627 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 60b27a84b518..fdec6e6b5aa4 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 14d489fd0974..99cfd5828fb1 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index caa041ed2a94..006996b10fa3 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index a02f0ad9fe68..0bde4ac9f024 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index 91ec7a824e93..617ec72a60f8 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 54ae0984239f..f274cddf2e83 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 122256f126be..9f20e6a21b6a 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index a2540c284da7..87fe9b529a3f 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.203-tw-0.50 + 0.203-tw-0.51 presto-matching diff --git a/presto-memory-context/pom.xml b/presto-memory-context/pom.xml index 44f49a9524e1..e67d91dd9d59 100644 --- a/presto-memory-context/pom.xml +++ b/presto-memory-context/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-memory-context diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 8ba2b32d7d37..a9c74f05b509 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index cd37b33ff496..c460544a6197 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index f108fe80099e..e07ce775bb9b 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 160e84772612..d3117c39721e 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index c21424bd9316..107b1086efaf 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 27363afee38c..86d7425f73d0 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-parser diff --git a/presto-password-authenticators/pom.xml b/presto-password-authenticators/pom.xml index db1f72b87227..f11e39327703 100644 --- a/presto-password-authenticators/pom.xml +++ b/presto-password-authenticators/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-password-authenticators diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index ce548f66396b..4ca566807f8c 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index fbe314fcc05b..3e7ed99691bb 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 050f14810548..44f03da5b3de 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.203-tw-0.50 + 0.203-tw-0.51 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index ae7d9859a3a7..a45e5a9d00eb 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 3a567eef7c1e..ddd2c40c4234 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 5f66bcd807f0..1d0080ada205 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 0ea4023a886a..1498af3c57d6 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index 0bef5ce03bb0..fdf9b40f6d18 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 7256f5bae4e1..9b7220b61ee8 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 5934c1061b89..d07a44d8e10e 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 4f3d52b59e13..061a1d23cd4a 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index abb3e549d2c2..7b9bfa2740a3 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index f44215d90365..2f0b424a9af3 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.203-tw-0.50 + 0.203-tw-0.51 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 062f6537c523..a123d88b1061 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 25808a6bb137..16cc5a0443fc 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 86cc42e57aec..68199003fba1 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.203-tw-0.50 + 0.203-tw-0.51 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index d7d41c1d8afe..127454506392 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index e922869522e0..bcde5461f0a1 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index f7721f792734..d208a8f509fc 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index d095c668a5ce..3ed0013445c2 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index f96aed06ed31..45a0120d6d81 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index 0802772c865c..587fc4e415e9 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-twitter-functions diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index 7ead242e55f8..9cb337cca493 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index ed49787fd4fe..426f5bef4251 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 6e8f229cd377..6e2303e49627 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.50 + 0.203-tw-0.51 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.203-tw-0.50 + 0.203-tw-0.51 provided From 94e86395e6bede0722bb7dbf286214cb8b9c1922 Mon Sep 17 00:00:00 2001 From: Hao Luo Date: Fri, 13 Jul 2018 10:19:08 -0700 Subject: [PATCH 289/331] Fix missing fields in thriftEvent in eventlistener (#161) --- .../QueryCompletedEventScriber.java | 74 +++++++++---------- 1 file changed, 37 insertions(+), 37 deletions(-) diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java index e6bfd714ad36..6e0d31fd3352 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java @@ -62,51 +62,51 @@ private static QueryCompletionEvent toThriftQueryCompletionEvent(QueryCompletedE QueryCompletionEvent thriftEvent = new com.twitter.presto.thriftjava.QueryCompletionEvent(); - thriftEvent.query_id = eventMetadata.getQueryId(); - thriftEvent.transaction_id = eventMetadata.getTransactionId().orElse(DASH); - thriftEvent.user = eventContext.getUser(); - thriftEvent.principal = eventContext.getPrincipal().orElse(DASH); - thriftEvent.source = eventContext.getSource().orElse(DASH); - thriftEvent.server_version = eventContext.getServerVersion(); - thriftEvent.environment = eventContext.getEnvironment(); - thriftEvent.catalog = eventContext.getCatalog().orElse(DASH); - thriftEvent.schema = eventContext.getSchema().orElse(DASH); + thriftEvent.setQuery_id(eventMetadata.getQueryId()); + thriftEvent.setTransaction_id(eventMetadata.getTransactionId().orElse(DASH)); + thriftEvent.setUser(eventContext.getUser()); + thriftEvent.setPrincipal(eventContext.getPrincipal().orElse(DASH)); + thriftEvent.setSource(eventContext.getSource().orElse(DASH)); + thriftEvent.setServer_version(eventContext.getServerVersion()); + thriftEvent.setEnvironment(eventContext.getEnvironment()); + thriftEvent.setCatalog(eventContext.getCatalog().orElse(DASH)); + thriftEvent.setSchema(eventContext.getSchema().orElse(DASH)); Map> queriedColumnsByTable = new HashMap>(); event.getIoMetadata().getInputs().forEach(input -> queriedColumnsByTable.put(String.format("%s.%s", input.getSchema(), input.getTable()), input.getColumns())); - thriftEvent.queried_columns_by_table = queriedColumnsByTable; - thriftEvent.remote_client_address = eventContext.getRemoteClientAddress().orElse(DASH); - thriftEvent.user_agent = eventContext.getUserAgent().orElse(DASH); - thriftEvent.query_state = QueryState.valueOf(eventMetadata.getQueryState()); - thriftEvent.uri = eventMetadata.getUri().toString(); - thriftEvent.query = eventMetadata.getQuery(); - thriftEvent.create_time_ms = event.getCreateTime().toEpochMilli(); - thriftEvent.execution_start_time_ms = event.getExecutionStartTime().toEpochMilli(); - thriftEvent.end_time_ms = event.getEndTime().toEpochMilli(); - thriftEvent.queued_time_ms = eventStat.getQueuedTime().toMillis(); - thriftEvent.query_wall_time_ms = eventStat.getWallTime().toMillis(); - thriftEvent.cumulative_memory_bytesecond = eventStat.getCumulativeMemory(); - thriftEvent.peak_memory_bytes = eventStat.getPeakTotalNonRevocableMemoryBytes(); - thriftEvent.cpu_time_ms = eventStat.getCpuTime().toMillis(); + thriftEvent.setQueried_columns_by_table(queriedColumnsByTable); + thriftEvent.setRemote_client_address(eventContext.getRemoteClientAddress().orElse(DASH)); + thriftEvent.setUser_agent(eventContext.getUserAgent().orElse(DASH)); + thriftEvent.setQuery_state(QueryState.valueOf(eventMetadata.getQueryState())); + thriftEvent.setUri(eventMetadata.getUri().toString()); + thriftEvent.setQuery(eventMetadata.getQuery()); + thriftEvent.setCreate_time_ms(event.getCreateTime().toEpochMilli()); + thriftEvent.setExecution_start_time_ms(event.getExecutionStartTime().toEpochMilli()); + thriftEvent.setEnd_time_ms(event.getEndTime().toEpochMilli()); + thriftEvent.setQueued_time_ms(eventStat.getQueuedTime().toMillis()); + thriftEvent.setQuery_wall_time_ms(eventStat.getWallTime().toMillis()); + thriftEvent.setCumulative_memory_bytesecond(eventStat.getCumulativeMemory()); + thriftEvent.setPeak_memory_bytes(eventStat.getPeakTotalNonRevocableMemoryBytes()); + thriftEvent.setCpu_time_ms(eventStat.getCpuTime().toMillis()); if (eventStat.getAnalysisTime().isPresent()) { - thriftEvent.analysis_time_ms = eventStat.getAnalysisTime().get().toMillis(); + thriftEvent.setAnalysis_time_ms(eventStat.getAnalysisTime().get().toMillis()); } if (eventStat.getDistributedPlanningTime().isPresent()) { - thriftEvent.distributed_planning_time_ms = eventStat.getDistributedPlanningTime().get().toMillis(); + thriftEvent.setDistributed_planning_time_ms(eventStat.getDistributedPlanningTime().get().toMillis()); } - thriftEvent.total_bytes = eventStat.getTotalBytes(); - thriftEvent.query_stages = QueryStatsHelper.getQueryStages(eventMetadata); - thriftEvent.operator_summaries = QueryStatsHelper.getOperatorSummaries(eventStat); - thriftEvent.total_rows = eventStat.getTotalRows(); - thriftEvent.splits = eventStat.getCompletedSplits(); + thriftEvent.setTotal_bytes(eventStat.getTotalBytes()); + thriftEvent.setQuery_stages(QueryStatsHelper.getQueryStages(eventMetadata)); + thriftEvent.setOperator_summaries(QueryStatsHelper.getOperatorSummaries(eventStat)); + thriftEvent.setTotal_rows(eventStat.getTotalRows()); + thriftEvent.setSplits(eventStat.getCompletedSplits()); if (event.getFailureInfo().isPresent()) { QueryFailureInfo eventFailureInfo = event.getFailureInfo().get(); - thriftEvent.error_code_id = eventFailureInfo.getErrorCode().getCode(); - thriftEvent.error_code_name = eventFailureInfo.getErrorCode().getName(); - thriftEvent.failure_type = eventFailureInfo.getFailureType().orElse(DASH); - thriftEvent.failure_message = eventFailureInfo.getFailureMessage().orElse(DASH); - thriftEvent.failure_task = eventFailureInfo.getFailureTask().orElse(DASH); - thriftEvent.failure_host = eventFailureInfo.getFailureHost().orElse(DASH); - thriftEvent.failures_json = eventFailureInfo.getFailuresJson(); + thriftEvent.setError_code_id(eventFailureInfo.getErrorCode().getCode()); + thriftEvent.setError_code_name(eventFailureInfo.getErrorCode().getName()); + thriftEvent.setFailure_type(eventFailureInfo.getFailureType().orElse(DASH)); + thriftEvent.setFailure_message(eventFailureInfo.getFailureMessage().orElse(DASH)); + thriftEvent.setFailure_task(eventFailureInfo.getFailureTask().orElse(DASH)); + thriftEvent.setFailure_host(eventFailureInfo.getFailureHost().orElse(DASH)); + thriftEvent.setFailures_json(eventFailureInfo.getFailuresJson()); } return thriftEvent; From 00fe2e546f2a64533a2a770abbc1347b46a53060 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 13 Jul 2018 15:36:25 -0700 Subject: [PATCH 290/331] Release 0.203-tw-0.52 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial-toolkit/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory-context/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-password-authenticators/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 58 files changed, 60 insertions(+), 60 deletions(-) diff --git a/pom.xml b/pom.xml index fd6a372540c6..2256c0cab633 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.203-tw-0.51 + 0.203-tw-0.52 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 0e9e73675825..3207c60f9483 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 02be0b4a6ad9..64df02623b77 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 251ce499962c..4a7e14defc9f 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 0369c817e12d..7cb5a963dc19 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index b53870e0c222..41e37482008e 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index b7105da7bbc1..cabf9c4c5663 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.203-tw-0.51 + 0.203-tw-0.52 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 488bcbf2fc0b..c0679e34a758 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 38244a3ceeea..ddc3e1adcb7a 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 2615db84983b..eeba4cc3cee8 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 7dcbebe1475b..2cd83fdb54b7 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 84409e86cd05..936ee04e9332 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 57e3ee9243b1..82bed38ac7b9 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index be9928d37bbe..a498ed7f2f23 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index 7ccca0a2dfd9..424ce9e9d42e 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index c4390b18057e..1aff44a82bb6 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 35303f4dd627..e59e4bbd5087 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index fdec6e6b5aa4..452266e7b1b9 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 99cfd5828fb1..2a3f413fcbb7 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 006996b10fa3..9f3f0a345160 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 0bde4ac9f024..141470182616 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index 617ec72a60f8..b00f45288803 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index f274cddf2e83..a873b2576fe0 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 9f20e6a21b6a..0d1a296fdbd6 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index 87fe9b529a3f..bdc8c977cff5 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.203-tw-0.51 + 0.203-tw-0.52 presto-matching diff --git a/presto-memory-context/pom.xml b/presto-memory-context/pom.xml index e67d91dd9d59..f34dcb76c70f 100644 --- a/presto-memory-context/pom.xml +++ b/presto-memory-context/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-memory-context diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index a9c74f05b509..10c9eb2a669c 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index c460544a6197..3a56bea3bf88 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index e07ce775bb9b..e27211f92210 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index d3117c39721e..af96735f8227 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 107b1086efaf..ec6991bc4196 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 86d7425f73d0..efe1f17f49b8 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-parser diff --git a/presto-password-authenticators/pom.xml b/presto-password-authenticators/pom.xml index f11e39327703..c3c316708c3e 100644 --- a/presto-password-authenticators/pom.xml +++ b/presto-password-authenticators/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-password-authenticators diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 4ca566807f8c..5a56068f548f 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 3e7ed99691bb..64cce605298f 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 44f03da5b3de..0f4dceeca045 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.203-tw-0.51 + 0.203-tw-0.52 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index a45e5a9d00eb..99eafcc60fb0 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index ddd2c40c4234..2d376ec8217b 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 1d0080ada205..54a5f4c3c3fc 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 1498af3c57d6..a7351028d93c 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index fdf9b40f6d18..9c3906e08ace 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 9b7220b61ee8..6cc1edf2ba9f 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index d07a44d8e10e..e3b1dc642a52 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 061a1d23cd4a..91b0444d5afd 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 7b9bfa2740a3..055b7b090cb3 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 2f0b424a9af3..3b29e1d3ac0a 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.203-tw-0.51 + 0.203-tw-0.52 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index a123d88b1061..0d6f92483ea6 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 16cc5a0443fc..c0109ca7a49f 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 68199003fba1..ac7fee3caa3a 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.203-tw-0.51 + 0.203-tw-0.52 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index 127454506392..cc54faa19633 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index bcde5461f0a1..d1a7881ee78a 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index d208a8f509fc..8814a2fa64eb 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index 3ed0013445c2..6239a31985c6 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 45a0120d6d81..ecfae4009996 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index 587fc4e415e9..891835bdc4ea 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-twitter-functions diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index 9cb337cca493..dcfa2f9afb13 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 426f5bef4251..39508eb4899a 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 6e2303e49627..45ac40506472 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.51 + 0.203-tw-0.52 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.203-tw-0.51 + 0.203-tw-0.52 provided From 0a2cf42a2243a57981c7c22f2b6d751aca568f19 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 22 Aug 2018 11:31:39 -0700 Subject: [PATCH 291/331] Change version number to 0.203 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial-toolkit/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory-context/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-password-authenticators/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 58 files changed, 60 insertions(+), 60 deletions(-) diff --git a/pom.xml b/pom.xml index 2256c0cab633..8d64919a1b0a 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.203-tw-0.52 + 0.203 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 3207c60f9483..87d2b8bd7813 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 64df02623b77..378de0eae2f4 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 4a7e14defc9f..3cc797f4c7c1 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 7cb5a963dc19..0b958bd4a2ee 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 41e37482008e..d54ac84ff0da 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index cabf9c4c5663..5d6b5168c247 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.203-tw-0.52 + 0.203 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index c0679e34a758..260b7d789a56 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index ddc3e1adcb7a..34ef8d41d496 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index eeba4cc3cee8..c5eb2039e5e4 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 2cd83fdb54b7..bf0a52108b93 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 936ee04e9332..539e594afe4c 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 82bed38ac7b9..ec41dd31d746 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index a498ed7f2f23..8a78c81ca64e 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index 424ce9e9d42e..87a4bff0c3ba 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index 1aff44a82bb6..91f529c72ed7 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index e59e4bbd5087..03c92129bdbf 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 452266e7b1b9..c30f3d9ed2af 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 2a3f413fcbb7..89064950e849 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 9f3f0a345160..989c08a26de9 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 141470182616..edc53df7a8e8 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index b00f45288803..af2f9bd2cdfd 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index a873b2576fe0..5e4800b0fcae 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 0d1a296fdbd6..bfd40b733e01 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index bdc8c977cff5..55c0b1e9a5a4 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.203-tw-0.52 + 0.203 presto-matching diff --git a/presto-memory-context/pom.xml b/presto-memory-context/pom.xml index f34dcb76c70f..31a664538393 100644 --- a/presto-memory-context/pom.xml +++ b/presto-memory-context/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-memory-context diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 10c9eb2a669c..8ce423171949 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 3a56bea3bf88..e2ed26b14409 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index e27211f92210..35f6de53af6e 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index af96735f8227..4ac0d9575a23 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index ec6991bc4196..caf7b344f1ce 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index efe1f17f49b8..a86e2222b42c 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-parser diff --git a/presto-password-authenticators/pom.xml b/presto-password-authenticators/pom.xml index c3c316708c3e..bed4c7871d41 100644 --- a/presto-password-authenticators/pom.xml +++ b/presto-password-authenticators/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-password-authenticators diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 5a56068f548f..a8ecabe333ac 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 64cce605298f..63efe4b9fcf6 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 0f4dceeca045..daec15262e90 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.203-tw-0.52 + 0.203 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 99eafcc60fb0..8fd17ddb8d4d 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 2d376ec8217b..2c81b8ba01b3 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 54a5f4c3c3fc..d6ae4f7e1b0b 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index a7351028d93c..d227b1bb41f8 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index 9c3906e08ace..c06f9727c15a 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 6cc1edf2ba9f..c6efca1acab3 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index e3b1dc642a52..0969ab4aae65 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 91b0444d5afd..b4ff169ed3e1 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 055b7b090cb3..7d5a257a68ab 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 3b29e1d3ac0a..cabc90930c1c 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.203-tw-0.52 + 0.203 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 0d6f92483ea6..2955728fcff8 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index c0109ca7a49f..7ee55814bdc5 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index ac7fee3caa3a..ff9418fe3f5e 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.203-tw-0.52 + 0.203 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index cc54faa19633..9baae2a32ff0 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index d1a7881ee78a..4684a5edddc1 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index 8814a2fa64eb..2769add60d29 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index 6239a31985c6..e934b34c8298 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index ecfae4009996..88d4fa686b0f 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index 891835bdc4ea..6ef704e8ab28 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-twitter-functions diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index dcfa2f9afb13..48eda2f04c58 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 39508eb4899a..07d801e8cd8b 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 45ac40506472..c787fa74bde3 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203-tw-0.52 + 0.203 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.203-tw-0.52 + 0.203 provided From b113fb5298ea9bf4a639a9c43783c1ad26c3235a Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 22 Aug 2018 11:35:57 -0700 Subject: [PATCH 292/331] Change version number to 0.208 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial-toolkit/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory-context/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-password-authenticators/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 58 files changed, 60 insertions(+), 60 deletions(-) diff --git a/pom.xml b/pom.xml index 8d64919a1b0a..15cfc6fb7434 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.203 + 0.208 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 87d2b8bd7813..70cd1ecb1891 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 378de0eae2f4..5a4161e21afd 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 3cc797f4c7c1..8b698ada9114 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 0b958bd4a2ee..4b5878480cb5 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index d54ac84ff0da..02c17107f4d0 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 5d6b5168c247..4fccc07c493b 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.203 + 0.208 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 260b7d789a56..38a11fdf62f5 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 34ef8d41d496..69f3024b98ce 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index c5eb2039e5e4..781aaea45240 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index bf0a52108b93..59650091a684 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 539e594afe4c..7c4b689d118f 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index ec41dd31d746..ee13a0daa3e3 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 8a78c81ca64e..92a4a600b227 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index 87a4bff0c3ba..b463822af58a 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index 91f529c72ed7..184e93c0fe80 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 03c92129bdbf..9c9620641de1 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index c30f3d9ed2af..752e33092b2c 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 89064950e849..330a45b9701d 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 989c08a26de9..87d63c976f03 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index edc53df7a8e8..b61c60e7381d 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index af2f9bd2cdfd..c43ea0b5cde3 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 5e4800b0fcae..600213db280c 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index bfd40b733e01..78cac594f920 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index 55c0b1e9a5a4..e8df6cfb48bc 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.203 + 0.208 presto-matching diff --git a/presto-memory-context/pom.xml b/presto-memory-context/pom.xml index 31a664538393..3c52d4e70404 100644 --- a/presto-memory-context/pom.xml +++ b/presto-memory-context/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-memory-context diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 8ce423171949..1bd12ce390b0 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index e2ed26b14409..ec72b39e5ead 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 35f6de53af6e..08c3f0c9fbf3 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 4ac0d9575a23..36f00f239d85 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index caf7b344f1ce..67e06d896d01 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index a86e2222b42c..57ac7c42100c 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-parser diff --git a/presto-password-authenticators/pom.xml b/presto-password-authenticators/pom.xml index bed4c7871d41..9b9529d80242 100644 --- a/presto-password-authenticators/pom.xml +++ b/presto-password-authenticators/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-password-authenticators diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index a8ecabe333ac..298f9accfebb 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 63efe4b9fcf6..d3fa627467a9 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index daec15262e90..a1a4b434a03e 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.203 + 0.208 presto-product-tests diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 8fd17ddb8d4d..e8a4ec7696e4 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 2c81b8ba01b3..ca9a530887f4 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index d6ae4f7e1b0b..bc783539d80e 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index d227b1bb41f8..6c67032c42c0 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index c06f9727c15a..40d504f9a655 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index c6efca1acab3..1811bca12648 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 0969ab4aae65..a6f65391b235 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index b4ff169ed3e1..c8e57e3d8344 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 7d5a257a68ab..51fc6a7910d3 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index cabc90930c1c..b7a01729ecee 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.203 + 0.208 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 2955728fcff8..b5da6617b75d 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 7ee55814bdc5..289898a63d03 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index ff9418fe3f5e..1208a06d2d30 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.203 + 0.208 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index 9baae2a32ff0..9617e782eb56 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 4684a5edddc1..4eca220613ce 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index 2769add60d29..866491aa426d 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index e934b34c8298..ab5ee3443801 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 88d4fa686b0f..f7c07e9392b5 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index 6ef704e8ab28..3e14c3da6325 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-twitter-functions diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index 48eda2f04c58..02697d4ecd2b 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 07d801e8cd8b..fb0a44cd4992 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index c787fa74bde3..b72489f73ac4 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.203 + 0.208 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.203 + 0.208 provided From 8587b9ca4fceda004592b09c5aca0b532cf930d6 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 22 Aug 2018 16:34:44 -0700 Subject: [PATCH 293/331] Update version 0.208-tw-0.53 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial-toolkit/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory-context/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-password-authenticators/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-proxy/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 59 files changed, 61 insertions(+), 61 deletions(-) diff --git a/pom.xml b/pom.xml index 94f550fd4f4c..ffbaac06cfaf 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.208 + 0.208-tw-0.53 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 70cd1ecb1891..0162bd07d566 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 5a4161e21afd..ed7347ac1355 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 8b698ada9114..7b6eab0adb5a 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 4b5878480cb5..fa4ba2f80ca4 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 02c17107f4d0..30da0a7ec2c2 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 4fccc07c493b..caad794c54e4 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.208 + 0.208-tw-0.53 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 38a11fdf62f5..916e95cf4c8d 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 69f3024b98ce..f308bdf1b432 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 781aaea45240..c03b54074df4 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 59650091a684..f5f67257a7d6 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 7c4b689d118f..9aa98505304d 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index ee13a0daa3e3..cefee42b89d1 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 92a4a600b227..9d416a6e0995 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index b463822af58a..0d95cfef8f6f 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index 184e93c0fe80..2f3379ca45d3 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 33c56808c5e6..b9b94fecfbd5 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 29aa4da1ef1a..833e7c639e79 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 330a45b9701d..75fecc836abf 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 87d63c976f03..19cef1cb9ae1 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index e971cf7561c1..dd8c20437276 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index c43ea0b5cde3..53363bd92c96 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 600213db280c..1671dfe836f5 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index e4d71c29b141..c885bd0491c2 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index e8df6cfb48bc..f51fc99922e2 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.208 + 0.208-tw-0.53 presto-matching diff --git a/presto-memory-context/pom.xml b/presto-memory-context/pom.xml index 3c52d4e70404..39c1506c76cb 100644 --- a/presto-memory-context/pom.xml +++ b/presto-memory-context/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-memory-context diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 1bd12ce390b0..3437b2bfe3ad 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index ec72b39e5ead..14c9eacfd8a9 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 08c3f0c9fbf3..44e992baf3a5 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 36f00f239d85..fe687e6ec226 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 67e06d896d01..9272b793dca6 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index b799cde2883f..cc74458d40ec 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-parser diff --git a/presto-password-authenticators/pom.xml b/presto-password-authenticators/pom.xml index 9b9529d80242..24299c6ecd74 100644 --- a/presto-password-authenticators/pom.xml +++ b/presto-password-authenticators/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-password-authenticators diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index faefc9fec5ce..3e89e93dab4a 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index d3fa627467a9..13c45340e3cc 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 8959c496eb5b..e2e394b43ed0 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.208 + 0.208-tw-0.53 presto-product-tests diff --git a/presto-proxy/pom.xml b/presto-proxy/pom.xml index 2ab17d70d97a..51748a035e83 100644 --- a/presto-proxy/pom.xml +++ b/presto-proxy/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-proxy diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index e8a4ec7696e4..eabad7a1e316 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index ca9a530887f4..343b060d5f4a 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 3f29f4c77b72..9ef329b09134 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 6c67032c42c0..9229224b4cb8 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index 40d504f9a655..7cf35f27a7d6 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 1811bca12648..b289a837ec37 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index a6f65391b235..86dd879c4437 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index c8e57e3d8344..1c1755b13d0d 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 51fc6a7910d3..b4c502de793a 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index b7a01729ecee..4f3318e112fa 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.208 + 0.208-tw-0.53 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index b5da6617b75d..c6e9c285ad54 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 289898a63d03..d2b62dda1bc3 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 1208a06d2d30..ff965e906b36 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.208 + 0.208-tw-0.53 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index 9617e782eb56..b5f6dc554b88 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 4eca220613ce..d6910d0c8fe7 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index 866491aa426d..fe3074a5fdb4 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index ab5ee3443801..2ccaa4d15b8a 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index f7c07e9392b5..533b3c6b1b15 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index 3e14c3da6325..38322e3cbcd4 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-twitter-functions diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index 02697d4ecd2b..c9b82b8eb432 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index fb0a44cd4992..ae7be871d377 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index b72489f73ac4..bf6ca6d422c4 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208 + 0.208-tw-0.53 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.208 + 0.208-tw-0.53 provided From 17ded159b6bc284b53bfac79574d5ee69f741f46 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 30 Aug 2018 15:39:45 -0700 Subject: [PATCH 294/331] Fix name based column lookup --- .../com/facebook/presto/hive/parquet/ParquetPageSource.java | 4 ++-- .../com/facebook/presto/hive/parquet/ParquetTypeUtils.java | 2 +- presto-hive/src/main/java/parquet/io/ColumnIOConverter.java | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSource.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSource.java index 70e88094d350..248f86f88ecb 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSource.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSource.java @@ -38,9 +38,9 @@ import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveErrorCode.HIVE_BAD_DATA; import static com.facebook.presto.hive.HiveErrorCode.HIVE_CURSOR_ERROR; +import static com.facebook.presto.hive.parquet.ParquetTypeUtils.findColumnIObyName; import static com.facebook.presto.hive.parquet.ParquetTypeUtils.findFieldIndexByName; import static com.facebook.presto.hive.parquet.ParquetTypeUtils.getParquetType; -import static com.facebook.presto.hive.parquet.ParquetTypeUtils.lookupColumnByName; import static com.google.common.base.Preconditions.checkState; import static java.util.Objects.requireNonNull; import static parquet.io.ColumnIOConverter.constructField; @@ -106,7 +106,7 @@ public ParquetPageSource( } else { String columnName = useParquetColumnNames ? name : fileSchema.getFields().get(column.getHiveColumnIndex()).getName(); - fieldsBuilder.add(constructField(type, lookupColumnByName(messageColumnIO, columnName))); + fieldsBuilder.add(constructField(type, findColumnIObyName(messageColumnIO, columnName))); } } types = typesBuilder.build(); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java index 027c498a0b30..5283e5736ff7 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetTypeUtils.java @@ -270,7 +270,7 @@ public static ColumnIO findColumnIObyName(GroupColumnIO groupColumnIO, String na ColumnIO columnIO = lookupColumnByName(groupColumnIO, name); if (columnIO == null && name.endsWith("_")) { - return findColumnIObyName(groupColumnIO, name.substring(0, name.length() - 1)); + columnIO = lookupColumnByName(groupColumnIO, name.substring(0, name.length() - 1)); } return columnIO; diff --git a/presto-hive/src/main/java/parquet/io/ColumnIOConverter.java b/presto-hive/src/main/java/parquet/io/ColumnIOConverter.java index ca05ba94e8b1..e657878e54f3 100644 --- a/presto-hive/src/main/java/parquet/io/ColumnIOConverter.java +++ b/presto-hive/src/main/java/parquet/io/ColumnIOConverter.java @@ -27,9 +27,9 @@ import java.util.Locale; import java.util.Optional; +import static com.facebook.presto.hive.parquet.ParquetTypeUtils.findColumnIObyName; import static com.facebook.presto.hive.parquet.ParquetTypeUtils.getArrayElementColumn; import static com.facebook.presto.hive.parquet.ParquetTypeUtils.getMapKeyValueColumn; -import static com.facebook.presto.hive.parquet.ParquetTypeUtils.lookupColumnByName; import static com.facebook.presto.spi.type.StandardTypes.ARRAY; import static com.facebook.presto.spi.type.StandardTypes.MAP; import static com.facebook.presto.spi.type.StandardTypes.ROW; @@ -61,7 +61,7 @@ public static Optional constructField(Type type, ColumnIO columnIO) for (int i = 0; i < fields.size(); i++) { NamedTypeSignature namedTypeSignature = fields.get(i).getNamedTypeSignature(); String name = namedTypeSignature.getName().get().toLowerCase(Locale.ENGLISH); - Optional field = constructField(parameters.get(i), lookupColumnByName(groupColumnIO, name)); + Optional field = constructField(parameters.get(i), findColumnIObyName(groupColumnIO, name)); structHasParameters |= field.isPresent(); fieldsBuilder.add(field); } From 026aceebe7c450dcf5bfdc6de3de876504068ab6 Mon Sep 17 00:00:00 2001 From: Mainak Ghosh Date: Wed, 12 Sep 2018 11:01:50 -0700 Subject: [PATCH 295/331] Using 0.210 version for better merge experience --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial-toolkit/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kafka07/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory-context/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-password-authenticators/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-proxy/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 59 files changed, 61 insertions(+), 61 deletions(-) diff --git a/pom.xml b/pom.xml index ffbaac06cfaf..4fc9634518df 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.208-tw-0.53 + 0.210 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 0162bd07d566..8bc63cf9b7fb 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index ed7347ac1355..26eb28f17ea1 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 7b6eab0adb5a..841132b0d2df 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index fa4ba2f80ca4..fae14de244eb 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 30da0a7ec2c2..da1c7bb0804a 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index caad794c54e4..2a9221b6fa86 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.208-tw-0.53 + 0.210 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 916e95cf4c8d..bffbeb1b7fa4 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index f308bdf1b432..63aa547279ac 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index c03b54074df4..dc38ebd4a541 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index f5f67257a7d6..aae4e6093c10 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 9aa98505304d..96b98481f50d 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index cefee42b89d1..1b3051ddedd7 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 9d416a6e0995..2531116738ec 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index 0d95cfef8f6f..70d4ddbea0e4 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index 2f3379ca45d3..6fff422f0d2b 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index b9b94fecfbd5..cdcadcc072c3 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 833e7c639e79..fa6a37b46efd 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 75fecc836abf..556aae848ef1 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 19cef1cb9ae1..1cd91957a299 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index dd8c20437276..04a0eb833e10 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-kafka diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index 53363bd92c96..b02f47573844 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-kafka07 diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 1671dfe836f5..4d9d8098325b 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index c885bd0491c2..999dd17d0cd0 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index f51fc99922e2..8a4c3941a91e 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.208-tw-0.53 + 0.210 presto-matching diff --git a/presto-memory-context/pom.xml b/presto-memory-context/pom.xml index 39c1506c76cb..7199f8ca5bb4 100644 --- a/presto-memory-context/pom.xml +++ b/presto-memory-context/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-memory-context diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 3437b2bfe3ad..ba3b35aaba70 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 14c9eacfd8a9..379a1bc9e5b6 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 44e992baf3a5..f7556658a284 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index fe687e6ec226..6833037f5544 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 9272b793dca6..f64ae452a3dc 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index cc74458d40ec..afcc9bea28e8 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-parser diff --git a/presto-password-authenticators/pom.xml b/presto-password-authenticators/pom.xml index 24299c6ecd74..c5adf1238ebc 100644 --- a/presto-password-authenticators/pom.xml +++ b/presto-password-authenticators/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-password-authenticators diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 3e89e93dab4a..8a9cde19f21d 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 13c45340e3cc..d0dcc83f0b46 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index e2e394b43ed0..d41b2b01f6a4 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.208-tw-0.53 + 0.210 presto-product-tests diff --git a/presto-proxy/pom.xml b/presto-proxy/pom.xml index 51748a035e83..74c58fb7147b 100644 --- a/presto-proxy/pom.xml +++ b/presto-proxy/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-proxy diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index eabad7a1e316..7df8064b391a 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 343b060d5f4a..0647cfc20a05 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 9ef329b09134..a5177b8834f2 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 9229224b4cb8..187a040263ff 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index 7cf35f27a7d6..ef78d6544de9 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index b289a837ec37..6a23989536f8 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 86dd879c4437..93d256900be1 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 1c1755b13d0d..b532bb148e84 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index b4c502de793a..8884605add52 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 4f3318e112fa..d6f7672625ad 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.208-tw-0.53 + 0.210 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index c6e9c285ad54..f0b661c81788 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index d2b62dda1bc3..9650ce61bf4d 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index ff965e906b36..603aa8de9489 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.208-tw-0.53 + 0.210 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index b5f6dc554b88..0696f76af869 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index d6910d0c8fe7..dcd44885fe4a 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index fe3074a5fdb4..bdee71399dc2 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index 2ccaa4d15b8a..0d22c82d411a 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 533b3c6b1b15..87ef177d7dc4 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index 38322e3cbcd4..f38792db72be 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-twitter-functions diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index c9b82b8eb432..556a931921aa 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index ae7be871d377..72bb0807fc5f 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index bf6ca6d422c4..e832a0406962 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.208-tw-0.53 + 0.210 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.208-tw-0.53 + 0.210 provided From 0d44e2ca14505ac6846d4b558ad13e9eec9b6a9d Mon Sep 17 00:00:00 2001 From: Mainak Ghosh Date: Wed, 12 Sep 2018 14:01:00 -0700 Subject: [PATCH 296/331] Fixing some of the compilation issues --- .../com/facebook/presto/decoder/FieldValueProvider.java | 5 ----- .../presto/twitter/functions/TwitterStringFunctions.java | 6 +++--- 2 files changed, 3 insertions(+), 8 deletions(-) diff --git a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/FieldValueProvider.java b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/FieldValueProvider.java index e7a537156d01..9c8e3b3f58bf 100644 --- a/presto-record-decoder/src/main/java/com/facebook/presto/decoder/FieldValueProvider.java +++ b/presto-record-decoder/src/main/java/com/facebook/presto/decoder/FieldValueProvider.java @@ -48,9 +48,4 @@ public Block getBlock() } public abstract boolean isNull(); - - public Block getBlock() - { - throw new PrestoException(DecoderErrorCode.DECODER_CONVERSION_NOT_SUPPORTED, "conversion to block not supported"); - } } diff --git a/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterStringFunctions.java b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterStringFunctions.java index 9f43baaafe3a..563682e56cb4 100644 --- a/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterStringFunctions.java +++ b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterStringFunctions.java @@ -38,7 +38,7 @@ private TwitterStringFunctions() @ScalarFunction("split_every") @Description("Splits the string on every character and returns an array") - @LiteralParameters({"x"}) + @LiteralParameters("x") @SqlType("array(varchar(x))") public static Block str2array(@SqlType("varchar(x)") Slice utf8) { @@ -47,7 +47,7 @@ public static Block str2array(@SqlType("varchar(x)") Slice utf8) @ScalarFunction("split_every") @Description("Splits the string on every given length of characters and returns an array") - @LiteralParameters({"x"}) + @LiteralParameters("x") @SqlType("array(varchar(x))") public static Block str2array(@SqlType("varchar(x)") Slice utf8, @SqlType(StandardTypes.BIGINT) long length) { @@ -56,7 +56,7 @@ public static Block str2array(@SqlType("varchar(x)") Slice utf8, @SqlType(Standa @ScalarFunction("split_every") @Description("Splits the string on every given length of characters and returns an array with the size at most of the given limit") - @LiteralParameters({"x"}) + @LiteralParameters("x") @SqlType("array(varchar(x))") public static Block str2array(@SqlType("varchar(x)") Slice utf8, @SqlType(StandardTypes.BIGINT) long length, @SqlType(StandardTypes.BIGINT) long limit) { From f89a436fd0f66b96799f83c7f8201f232840ab3d Mon Sep 17 00:00:00 2001 From: Mainak Ghosh Date: Thu, 13 Sep 2018 14:54:41 -0700 Subject: [PATCH 297/331] Increasing travis_wait duration to resolve travis test timeout --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 832295e5e936..1054d8d85a50 100644 --- a/.travis.yml +++ b/.travis.yml @@ -96,7 +96,7 @@ script: fi - | if [[ -v TEST_SPECIFIC_MODULES ]]; then - ./mvnw test $MAVEN_SKIP_CHECKS_AND_DOCS -P !twitter-modules -B -pl $TEST_SPECIFIC_MODULES $TEST_FLAGS + travis_wait 60 ./mvnw test $MAVEN_SKIP_CHECKS_AND_DOCS -P !twitter-modules -B -pl $TEST_SPECIFIC_MODULES $TEST_FLAGS fi - | if [[ -v TEST_OTHER_MODULES ]]; then From a5cf2ff110d7b2ef549f6af1e64c7bc7b4c2e886 Mon Sep 17 00:00:00 2001 From: Hao Luo Date: Tue, 11 Sep 2018 12:06:41 -0700 Subject: [PATCH 298/331] remove presto-kafka07 module (cherry picked from commit 684f9cd) --- pom.xml | 1 - presto-kafka07/pom.xml | 244 ------------ .../presto/kafka/KafkaColumnHandle.java | 224 ----------- .../facebook/presto/kafka/KafkaConnector.java | 92 ----- .../presto/kafka/KafkaConnectorConfig.java | 204 ---------- .../presto/kafka/KafkaConnectorFactory.java | 95 ----- .../presto/kafka/KafkaConnectorId.java | 53 --- .../presto/kafka/KafkaConnectorModule.java | 78 ---- .../facebook/presto/kafka/KafkaErrorCode.java | 42 --- .../presto/kafka/KafkaHandleResolver.java | 89 ----- .../kafka/KafkaInternalFieldDescription.java | 153 -------- .../facebook/presto/kafka/KafkaMetadata.java | 274 -------------- .../facebook/presto/kafka/KafkaPlugin.java | 47 --- .../facebook/presto/kafka/KafkaRecordSet.java | 353 ------------------ .../presto/kafka/KafkaRecordSetProvider.java | 80 ---- .../kafka/KafkaSimpleConsumerManager.java | 103 ----- .../com/facebook/presto/kafka/KafkaSplit.java | 165 -------- .../presto/kafka/KafkaSplitManager.java | 153 -------- .../kafka/KafkaTableDescriptionSupplier.java | 138 ------- .../presto/kafka/KafkaTableHandle.java | 151 -------- .../presto/kafka/KafkaTableLayoutHandle.java | 64 ---- .../presto/kafka/KafkaTopicDescription.java | 92 ----- .../kafka/KafkaTopicFieldDescription.java | 159 -------- .../presto/kafka/KafkaTopicFieldGroup.java | 62 --- .../presto/kafka/KafkaTransactionHandle.java | 22 -- .../com/facebook/presto/kafka/KafkaUtil.java | 159 -------- .../presto/kafka/KafkaQueryRunner.java | 142 ------- .../kafka/TestKafkaConnectorConfig.java | 68 ---- .../presto/kafka/TestKafkaDistributed.java | 52 --- .../kafka/TestKafkaIntegrationSmokeTest.java | 52 --- .../presto/kafka/TestKafkaPlugin.java | 46 --- .../presto/kafka/TestManySegments.java | 118 ------ .../kafka/TestMinimalFunctionality.java | 147 -------- .../presto/kafka/util/CodecSupplier.java | 71 ---- .../presto/kafka/util/EmbeddedKafka.java | 166 -------- .../presto/kafka/util/EmbeddedZookeeper.java | 101 ----- .../presto/kafka/util/JsonEncoder.java | 39 -- .../presto/kafka/util/KafkaLoader.java | 158 -------- .../presto/kafka/util/NumberEncoder.java | 31 -- .../presto/kafka/util/NumberPartitioner.java | 33 -- .../facebook/presto/kafka/util/TestUtils.java | 96 ----- .../src/test/resources/tpch/customer.json | 61 --- .../src/test/resources/tpch/lineitem.json | 104 ------ .../src/test/resources/tpch/nation.json | 41 -- .../src/test/resources/tpch/orders.json | 67 ---- .../src/test/resources/tpch/part.json | 66 ---- .../src/test/resources/tpch/partsupp.json | 46 --- .../src/test/resources/tpch/region.json | 36 -- .../src/test/resources/tpch/supplier.json | 56 --- presto-main/etc/config.properties | 1 - .../src/main/provisio/twitter.xml | 6 - 51 files changed, 5101 deletions(-) delete mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaColumnHandle.java delete mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnector.java delete mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorConfig.java delete mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorFactory.java delete mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorId.java delete mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaErrorCode.java delete mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaHandleResolver.java delete mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaPlugin.java delete mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSimpleConsumerManager.java delete mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplit.java delete mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java delete mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableDescriptionSupplier.java delete mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableHandle.java delete mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableLayoutHandle.java delete mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicDescription.java delete mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicFieldDescription.java delete mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicFieldGroup.java delete mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTransactionHandle.java delete mode 100644 presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaUtil.java delete mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/KafkaQueryRunner.java delete mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaConnectorConfig.java delete mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaDistributed.java delete mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaIntegrationSmokeTest.java delete mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaPlugin.java delete mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/TestManySegments.java delete mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/TestMinimalFunctionality.java delete mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/util/CodecSupplier.java delete mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedKafka.java delete mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedZookeeper.java delete mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/util/JsonEncoder.java delete mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberEncoder.java delete mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberPartitioner.java delete mode 100644 presto-kafka07/src/test/java/com/facebook/presto/kafka/util/TestUtils.java delete mode 100644 presto-kafka07/src/test/resources/tpch/customer.json delete mode 100644 presto-kafka07/src/test/resources/tpch/lineitem.json delete mode 100644 presto-kafka07/src/test/resources/tpch/nation.json delete mode 100644 presto-kafka07/src/test/resources/tpch/orders.json delete mode 100644 presto-kafka07/src/test/resources/tpch/part.json delete mode 100644 presto-kafka07/src/test/resources/tpch/partsupp.json delete mode 100644 presto-kafka07/src/test/resources/tpch/region.json delete mode 100644 presto-kafka07/src/test/resources/tpch/supplier.json diff --git a/pom.xml b/pom.xml index 0df82ee68b0e..a553a77f7d37 100644 --- a/pom.xml +++ b/pom.xml @@ -1300,7 +1300,6 @@ true - presto-kafka07 twitter-eventlistener-plugin presto-twitter-server diff --git a/presto-kafka07/pom.xml b/presto-kafka07/pom.xml index 34cc68673876..e69de29bb2d1 100644 --- a/presto-kafka07/pom.xml +++ b/presto-kafka07/pom.xml @@ -1,244 +0,0 @@ - - - 4.0.0 - - - com.facebook.presto - presto-root - 0.210-tw-0.54 - - - presto-kafka07 - Presto - Kafka Connector for ver0.7 - presto-plugin - - - ${project.parent.basedir} - - - true - - - - - io.airlift - bootstrap - - - - io.airlift - json - - - - io.airlift - log - - - - io.airlift - configuration - - - - com.facebook.presto - presto-record-decoder - - - - com.google.guava - guava - - - - com.google.inject - guice - - - - javax.validation - validation-api - - - - com.twitter - rosette-kafka_2.11 - 0.7.2-21 - - - jsr305 - com.google.code.findbugs - - - zookeeper - org.apache.zookeeper - - - finagle-ostrich4_2.10 - com.twitter - - - commons-lang - commons-lang - - - - - - org.apache.zookeeper - zookeeper - - - - - joda-time - joda-time - - - - org.scala-lang - scala-library - 2.11.7 - - - - javax.annotation - javax.annotation-api - - - - javax.inject - javax.inject - - - - com.fasterxml.jackson.core - jackson-databind - - - - - com.facebook.presto - presto-spi - provided - - - - io.airlift - slice - provided - - - - io.airlift - units - provided - - - - com.fasterxml.jackson.core - jackson-annotations - provided - - - - - io.airlift - log-manager - runtime - - - - - org.testng - testng - test - - - - io.airlift - testing - test - - - - com.facebook.presto - presto-main - test - - - - com.facebook.presto - presto-tpch - test - - - - com.facebook.presto - presto-client - test - - - - com.facebook.presto - presto-tests - test - - - - io.airlift.tpch - tpch - test - - - - com.github.sgroschupf - zkclient - 0.1 - - - log4j - log4j - - - - - - org.jetbrains - annotations - test - - - - - - - org.apache.maven.plugins - maven-surefire-plugin - - - - **/TestKafkaDistributed.java - - - - - - - - - ci - - - - org.apache.maven.plugins - maven-surefire-plugin - - - - - - - - - diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaColumnHandle.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaColumnHandle.java deleted file mode 100644 index b8ec023b2401..000000000000 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaColumnHandle.java +++ /dev/null @@ -1,224 +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.kafka; - -import com.facebook.presto.decoder.DecoderColumnHandle; -import com.facebook.presto.spi.ColumnMetadata; -import com.facebook.presto.spi.type.Type; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.util.Objects; - -import static com.google.common.base.MoreObjects.toStringHelper; -import static java.util.Objects.requireNonNull; - -/** - * Kafka specific connector column handle. - */ -public final class KafkaColumnHandle - implements DecoderColumnHandle, Comparable -{ - private final String connectorId; - private final int ordinalPosition; - - /** - * Column Name - */ - private final String name; - - /** - * Column type - */ - private final Type type; - - /** - * Mapping hint for the decoder. Can be null. - */ - private final String mapping; - - /** - * Data format to use (selects the decoder). Can be null. - */ - private final String dataFormat; - - /** - * Additional format hint for the selected decoder. Selects a decoder subtype (e.g. which timestamp decoder). - */ - private final String formatHint; - - /** - * True if the key decoder should be used, false if the message decoder should be used. - */ - private final boolean keyDecoder; - - /** - * True if the column should be hidden. - */ - private final boolean hidden; - - /** - * True if the column is internal to the connector and not defined by a topic definition. - */ - private final boolean internal; - - @JsonCreator - public KafkaColumnHandle( - @JsonProperty("connectorId") String connectorId, - @JsonProperty("ordinalPosition") int ordinalPosition, - @JsonProperty("name") String name, - @JsonProperty("type") Type type, - @JsonProperty("mapping") String mapping, - @JsonProperty("dataFormat") String dataFormat, - @JsonProperty("formatHint") String formatHint, - @JsonProperty("keyDecoder") boolean keyDecoder, - @JsonProperty("hidden") boolean hidden, - @JsonProperty("internal") boolean internal) - - { - this.connectorId = requireNonNull(connectorId, "connectorId is null"); - this.ordinalPosition = ordinalPosition; - this.name = requireNonNull(name, "name is null"); - this.type = requireNonNull(type, "type is null"); - this.mapping = mapping; - this.dataFormat = dataFormat; - this.formatHint = formatHint; - this.keyDecoder = keyDecoder; - this.hidden = hidden; - this.internal = internal; - } - - @JsonProperty - public String getConnectorId() - { - return connectorId; - } - - @JsonProperty - public int getOrdinalPosition() - { - return ordinalPosition; - } - - @Override - @JsonProperty - public String getName() - { - return name; - } - - @Override - @JsonProperty - public Type getType() - { - return type; - } - - @Override - @JsonProperty - public String getMapping() - { - return mapping; - } - - @Override - @JsonProperty - public String getDataFormat() - { - return dataFormat; - } - - @Override - @JsonProperty - public String getFormatHint() - { - return formatHint; - } - - @JsonProperty - public boolean isKeyDecoder() - { - return keyDecoder; - } - - @JsonProperty - public boolean isHidden() - { - return hidden; - } - - @Override - @JsonProperty - public boolean isInternal() - { - return internal; - } - - ColumnMetadata getColumnMetadata() - { - return new ColumnMetadata(name, type, null, hidden); - } - - @Override - public int hashCode() - { - return Objects.hash(connectorId, ordinalPosition, name, type, mapping, dataFormat, formatHint, keyDecoder, hidden, internal); - } - - @Override - public boolean equals(Object obj) - { - if (this == obj) { - return true; - } - if (obj == null || getClass() != obj.getClass()) { - return false; - } - - KafkaColumnHandle other = (KafkaColumnHandle) obj; - return Objects.equals(this.connectorId, other.connectorId) && - Objects.equals(this.ordinalPosition, other.ordinalPosition) && - Objects.equals(this.name, other.name) && - Objects.equals(this.type, other.type) && - Objects.equals(this.mapping, other.mapping) && - Objects.equals(this.dataFormat, other.dataFormat) && - Objects.equals(this.formatHint, other.formatHint) && - Objects.equals(this.keyDecoder, other.keyDecoder) && - Objects.equals(this.hidden, other.hidden) && - Objects.equals(this.internal, other.internal); - } - - @Override - public int compareTo(KafkaColumnHandle otherHandle) - { - return Integer.compare(this.getOrdinalPosition(), otherHandle.getOrdinalPosition()); - } - - @Override - public String toString() - { - return toStringHelper(this) - .add("connectorId", connectorId) - .add("ordinalPosition", ordinalPosition) - .add("name", name) - .add("type", type) - .add("mapping", mapping) - .add("dataFormat", dataFormat) - .add("formatHint", formatHint) - .add("keyDecoder", keyDecoder) - .add("hidden", hidden) - .add("internal", internal) - .toString(); - } -} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnector.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnector.java deleted file mode 100644 index 69cfe62e8737..000000000000 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnector.java +++ /dev/null @@ -1,92 +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.kafka; - -import com.facebook.presto.spi.connector.Connector; -import com.facebook.presto.spi.connector.ConnectorMetadata; -import com.facebook.presto.spi.connector.ConnectorRecordSetProvider; -import com.facebook.presto.spi.connector.ConnectorSplitManager; -import com.facebook.presto.spi.connector.ConnectorTransactionHandle; -import com.facebook.presto.spi.transaction.IsolationLevel; -import io.airlift.bootstrap.LifeCycleManager; -import io.airlift.log.Logger; - -import javax.inject.Inject; - -import static com.facebook.presto.spi.transaction.IsolationLevel.READ_COMMITTED; -import static com.facebook.presto.spi.transaction.IsolationLevel.checkConnectorSupports; -import static java.util.Objects.requireNonNull; - -/** - * Kafka specific implementation of the Presto Connector SPI. This is a read only connector. - */ -public class KafkaConnector - implements Connector -{ - private static final Logger log = Logger.get(KafkaConnector.class); - - private final LifeCycleManager lifeCycleManager; - private final KafkaMetadata metadata; - private final KafkaSplitManager splitManager; - private final KafkaRecordSetProvider recordSetProvider; - - @Inject - public KafkaConnector( - LifeCycleManager lifeCycleManager, - KafkaMetadata metadata, - KafkaSplitManager splitManager, - KafkaRecordSetProvider recordSetProvider) - { - this.lifeCycleManager = requireNonNull(lifeCycleManager, "lifeCycleManager is null"); - this.metadata = requireNonNull(metadata, "metadata is null"); - this.splitManager = requireNonNull(splitManager, "splitManager is null"); - this.recordSetProvider = requireNonNull(recordSetProvider, "recordSetProvider is null"); - } - - @Override - public ConnectorTransactionHandle beginTransaction(IsolationLevel isolationLevel, boolean readOnly) - { - checkConnectorSupports(READ_COMMITTED, isolationLevel); - return KafkaTransactionHandle.INSTANCE; - } - - @Override - public ConnectorMetadata getMetadata(ConnectorTransactionHandle transactionHandle) - { - return metadata; - } - - @Override - public ConnectorSplitManager getSplitManager() - { - return splitManager; - } - - @Override - public ConnectorRecordSetProvider getRecordSetProvider() - { - return recordSetProvider; - } - - @Override - public final void shutdown() - { - try { - lifeCycleManager.stop(); - } - catch (Exception e) { - log.error(e, "Error shutting down connector"); - } - } -} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorConfig.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorConfig.java deleted file mode 100644 index de5615de2167..000000000000 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorConfig.java +++ /dev/null @@ -1,204 +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.kafka; - -import com.facebook.presto.spi.HostAddress; -import com.google.common.base.Splitter; -import com.google.common.collect.ImmutableSet; -import io.airlift.configuration.Config; -import io.airlift.units.DataSize; -import io.airlift.units.DataSize.Unit; -import io.airlift.units.Duration; -import io.airlift.units.MinDuration; - -import javax.validation.constraints.NotNull; - -import java.io.File; -import java.util.Set; - -import static com.google.common.collect.Iterables.transform; - -public class KafkaConnectorConfig -{ - private static final int KAFKA_DEFAULT_PORT = 9092; - - /** - * Seed nodes for Kafka cluster. At least one must exist. - */ - private Set nodes = ImmutableSet.of(); - - /** - * Timeout to connect to Kafka. - */ - private Duration kafkaConnectTimeout = Duration.valueOf("10s"); - - /** - * Buffer size for connecting to Kafka. - */ - private DataSize kafkaBufferSize = new DataSize(64, Unit.KILOBYTE); - - /** - * The schema name to use in the connector. - */ - private String defaultSchema = "default"; - - /** - * Set of tables known to this connector. For each table, a description file may be present in the catalog folder which describes columns for the given topic. - */ - private Set tableNames = ImmutableSet.of(); - - /** - * Folder holding the JSON description files for Kafka topics. - */ - private File tableDescriptionDir = new File("etc/kafka07/"); - - /** - * Whether internal columns are shown in table metadata or not. Default is no. - */ - private boolean hideInternalColumns = true; - - /** - * ZK endpoint for getting broker list - */ - private String zkEndpoint = ""; - - /** - * Fetch size - */ - private int fetchSize = 10 * 1024 * 1024; - - @NotNull - public File getTableDescriptionDir() - { - return tableDescriptionDir; - } - - @Config("kafka.table-description-dir") - public KafkaConnectorConfig setTableDescriptionDir(File tableDescriptionDir) - { - this.tableDescriptionDir = tableDescriptionDir; - return this; - } - - @NotNull - public Set getTableNames() - { - return tableNames; - } - - @Config("kafka.table-names") - public KafkaConnectorConfig setTableNames(String tableNames) - { - this.tableNames = ImmutableSet.copyOf(Splitter.on(',').omitEmptyStrings().trimResults().split(tableNames)); - return this; - } - - @NotNull - public String getDefaultSchema() - { - return defaultSchema; - } - - @Config("kafka.default-schema") - public KafkaConnectorConfig setDefaultSchema(String defaultSchema) - { - this.defaultSchema = defaultSchema; - return this; - } - - public Set getNodes() - { - return nodes; - } - - @Config("kafka.nodes") - public KafkaConnectorConfig setNodes(String nodes) - { - this.nodes = (nodes == null) ? null : parseNodes(nodes); - return this; - } - - @MinDuration("1s") - public Duration getKafkaConnectTimeout() - { - return kafkaConnectTimeout; - } - - @Config("kafka.connect-timeout") - public KafkaConnectorConfig setKafkaConnectTimeout(String kafkaConnectTimeout) - { - this.kafkaConnectTimeout = Duration.valueOf(kafkaConnectTimeout); - return this; - } - - public DataSize getKafkaBufferSize() - { - return kafkaBufferSize; - } - - @Config("kafka.buffer-size") - public KafkaConnectorConfig setKafkaBufferSize(String kafkaBufferSize) - { - this.kafkaBufferSize = DataSize.valueOf(kafkaBufferSize); - return this; - } - - public boolean isHideInternalColumns() - { - return hideInternalColumns; - } - - @Config("kafka.hide-internal-columns") - public KafkaConnectorConfig setHideInternalColumns(boolean hideInternalColumns) - { - this.hideInternalColumns = hideInternalColumns; - return this; - } - - @NotNull - public String getZkEndpoint() - { - return zkEndpoint; - } - - @Config("kafka.zk-endpoint") - public KafkaConnectorConfig setZkEndpoint(String zkEndpoint) - { - this.zkEndpoint = zkEndpoint; - return this; - } - - public int getFetchSize() - { - return fetchSize; - } - - @Config("kafka.fetch-size") - public KafkaConnectorConfig setFetchSize(int fetchSize) - { - this.fetchSize = fetchSize; - return this; - } - - public static ImmutableSet parseNodes(String nodes) - { - Splitter splitter = Splitter.on(',').omitEmptyStrings().trimResults(); - return ImmutableSet.copyOf(transform(splitter.split(nodes), KafkaConnectorConfig::toHostAddress)); - } - - private static HostAddress toHostAddress(String value) - { - return HostAddress.fromString(value).withDefaultPort(KAFKA_DEFAULT_PORT); - } -} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorFactory.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorFactory.java deleted file mode 100644 index b9721f0459f8..000000000000 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorFactory.java +++ /dev/null @@ -1,95 +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.kafka; - -import com.facebook.presto.spi.ConnectorHandleResolver; -import com.facebook.presto.spi.NodeManager; -import com.facebook.presto.spi.SchemaTableName; -import com.facebook.presto.spi.connector.Connector; -import com.facebook.presto.spi.connector.ConnectorContext; -import com.facebook.presto.spi.connector.ConnectorFactory; -import com.facebook.presto.spi.type.TypeManager; -import com.google.common.base.Throwables; -import com.google.inject.Injector; -import com.google.inject.Scopes; -import com.google.inject.TypeLiteral; -import io.airlift.bootstrap.Bootstrap; -import io.airlift.json.JsonModule; - -import java.util.Map; -import java.util.Optional; -import java.util.function.Supplier; - -import static java.util.Objects.requireNonNull; - -/** - * Creates Kafka Connectors based off connectorId and specific configuration. - */ -public class KafkaConnectorFactory - implements ConnectorFactory -{ - private final Optional>> tableDescriptionSupplier; - - KafkaConnectorFactory(Optional>> tableDescriptionSupplier) - { - this.tableDescriptionSupplier = requireNonNull(tableDescriptionSupplier, "tableDescriptionSupplier is null"); - } - - @Override - public String getName() - { - return "kafka07"; - } - - @Override - public ConnectorHandleResolver getHandleResolver() - { - return new KafkaHandleResolver(); - } - - @Override - public Connector create(String connectorId, Map config, ConnectorContext context) - { - requireNonNull(connectorId, "connectorId is null"); - requireNonNull(config, "config is null"); - - try { - Bootstrap app = new Bootstrap( - new JsonModule(), - new KafkaConnectorModule(), - binder -> { - binder.bind(KafkaConnectorId.class).toInstance(new KafkaConnectorId(connectorId)); - binder.bind(TypeManager.class).toInstance(context.getTypeManager()); - binder.bind(NodeManager.class).toInstance(context.getNodeManager()); - - if (tableDescriptionSupplier.isPresent()) { - binder.bind(new TypeLiteral>>() {}).toInstance(tableDescriptionSupplier.get()); - } - else { - binder.bind(new TypeLiteral>>() {}).to(KafkaTableDescriptionSupplier.class).in(Scopes.SINGLETON); - } - }); - - Injector injector = app.strictConfig() - .doNotInitializeLogging() - .setRequiredConfigurationProperties(config) - .initialize(); - - return injector.getInstance(KafkaConnector.class); - } - catch (Exception e) { - throw Throwables.propagate(e); - } - } -} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorId.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorId.java deleted file mode 100644 index 3470980df073..000000000000 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorId.java +++ /dev/null @@ -1,53 +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.kafka; - -import java.util.Objects; - -import static java.util.Objects.requireNonNull; - -public class KafkaConnectorId -{ - private final String connectorId; - - public KafkaConnectorId(String connectorId) - { - this.connectorId = requireNonNull(connectorId, "connectorId is null"); - } - - @Override - public boolean equals(Object obj) - { - if (this == obj) { - return true; - } - if (obj == null || getClass() != obj.getClass()) { - return false; - } - KafkaConnectorId other = (KafkaConnectorId) obj; - return Objects.equals(this.connectorId, other.connectorId); - } - - @Override - public int hashCode() - { - return Objects.hash(connectorId); - } - - @Override - public String toString() - { - return connectorId; - } -} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorModule.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorModule.java index 8afa5068eb82..e69de29bb2d1 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorModule.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaConnectorModule.java @@ -1,78 +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.kafka; - -import com.facebook.presto.decoder.DecoderModule; -import com.facebook.presto.spi.type.Type; -import com.facebook.presto.spi.type.TypeManager; -import com.fasterxml.jackson.databind.DeserializationContext; -import com.fasterxml.jackson.databind.deser.std.FromStringDeserializer; -import com.google.inject.Binder; -import com.google.inject.Module; -import com.google.inject.Scopes; - -import javax.inject.Inject; - -import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature; -import static io.airlift.configuration.ConfigBinder.configBinder; -import static io.airlift.json.JsonBinder.jsonBinder; -import static io.airlift.json.JsonCodecBinder.jsonCodecBinder; -import static java.util.Objects.requireNonNull; - -/** - * Guice module for the Apache Kafka connector. - */ -public class KafkaConnectorModule - implements Module -{ - @Override - public void configure(Binder binder) - { - binder.bind(KafkaConnector.class).in(Scopes.SINGLETON); - - binder.bind(KafkaMetadata.class).in(Scopes.SINGLETON); - binder.bind(KafkaSplitManager.class).in(Scopes.SINGLETON); - binder.bind(KafkaRecordSetProvider.class).in(Scopes.SINGLETON); - - binder.bind(KafkaSimpleConsumerManager.class).in(Scopes.SINGLETON); - - configBinder(binder).bindConfig(KafkaConnectorConfig.class); - - jsonBinder(binder).addDeserializerBinding(Type.class).to(TypeDeserializer.class); - jsonCodecBinder(binder).bindJsonCodec(KafkaTopicDescription.class); - - binder.install(new DecoderModule()); - } - - public static final class TypeDeserializer - extends FromStringDeserializer - { - private static final long serialVersionUID = 1L; - - private final TypeManager typeManager; - - @Inject - public TypeDeserializer(TypeManager typeManager) - { - super(Type.class); - this.typeManager = requireNonNull(typeManager, "typeManager is null"); - } - - @Override - protected Type _deserialize(String value, DeserializationContext context) - { - return typeManager.getType(parseTypeSignature(value)); - } - } -} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaErrorCode.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaErrorCode.java deleted file mode 100644 index 9338b9849211..000000000000 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaErrorCode.java +++ /dev/null @@ -1,42 +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.kafka; - -import com.facebook.presto.spi.ErrorCode; -import com.facebook.presto.spi.ErrorCodeSupplier; -import com.facebook.presto.spi.ErrorType; - -import static com.facebook.presto.spi.ErrorType.EXTERNAL; - -/** - * Kafka connector specific error codes. - */ -public enum KafkaErrorCode - implements ErrorCodeSupplier -{ - KAFKA_SPLIT_ERROR(0, EXTERNAL); - - private final ErrorCode errorCode; - - KafkaErrorCode(int code, ErrorType type) - { - errorCode = new ErrorCode(code + 0x0102_0000, name(), type); - } - - @Override - public ErrorCode toErrorCode() - { - return errorCode; - } -} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaHandleResolver.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaHandleResolver.java deleted file mode 100644 index 539b08270129..000000000000 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaHandleResolver.java +++ /dev/null @@ -1,89 +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.kafka; - -import com.facebook.presto.spi.ColumnHandle; -import com.facebook.presto.spi.ConnectorHandleResolver; -import com.facebook.presto.spi.ConnectorSplit; -import com.facebook.presto.spi.ConnectorTableHandle; -import com.facebook.presto.spi.ConnectorTableLayoutHandle; -import com.facebook.presto.spi.connector.ConnectorTransactionHandle; - -import static com.google.common.base.Preconditions.checkArgument; -import static java.util.Objects.requireNonNull; - -/** - * Kafka specific {@link com.facebook.presto.spi.ConnectorHandleResolver} implementation. - */ -public class KafkaHandleResolver - implements ConnectorHandleResolver -{ - @Override - public Class getTableHandleClass() - { - return KafkaTableHandle.class; - } - - @Override - public Class getColumnHandleClass() - { - return KafkaColumnHandle.class; - } - - @Override - public Class getSplitClass() - { - return KafkaSplit.class; - } - - @Override - public Class getTableLayoutHandleClass() - { - return KafkaTableLayoutHandle.class; - } - - @Override - public Class getTransactionHandleClass() - { - return KafkaTransactionHandle.class; - } - - static KafkaTableHandle convertTableHandle(ConnectorTableHandle tableHandle) - { - requireNonNull(tableHandle, "tableHandle is null"); - checkArgument(tableHandle instanceof KafkaTableHandle, "tableHandle is not an instance of KafkaTableHandle"); - return (KafkaTableHandle) tableHandle; - } - - static KafkaColumnHandle convertColumnHandle(ColumnHandle columnHandle) - { - requireNonNull(columnHandle, "columnHandle is null"); - checkArgument(columnHandle instanceof KafkaColumnHandle, "columnHandle is not an instance of KafkaColumnHandle"); - return (KafkaColumnHandle) columnHandle; - } - - static KafkaSplit convertSplit(ConnectorSplit split) - { - requireNonNull(split, "split is null"); - checkArgument(split instanceof KafkaSplit, "split is not an instance of KafkaSplit"); - return (KafkaSplit) split; - } - - static KafkaTableLayoutHandle convertLayout(ConnectorTableLayoutHandle layout) - { - requireNonNull(layout, "layout is null"); - checkArgument(layout instanceof KafkaTableLayoutHandle, "layout is not an instance of KafkaTableLayoutHandle"); - return (KafkaTableLayoutHandle) layout; - } -} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaInternalFieldDescription.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaInternalFieldDescription.java index fb74806e0027..e69de29bb2d1 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaInternalFieldDescription.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaInternalFieldDescription.java @@ -1,153 +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.kafka; - -import com.facebook.presto.spi.ColumnMetadata; -import com.facebook.presto.spi.type.BigintType; -import com.facebook.presto.spi.type.BooleanType; -import com.facebook.presto.spi.type.Type; - -import java.util.Map; - -import static com.facebook.presto.spi.type.VarcharType.createUnboundedVarcharType; -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Strings.isNullOrEmpty; -import static com.google.common.collect.ImmutableMap.toImmutableMap; -import static java.util.Arrays.stream; -import static java.util.Objects.requireNonNull; -import static java.util.function.Function.identity; - -/** - * Describes an internal (managed by the connector) field which is added to each table row. The definition itself makes the row - * show up in the tables (the columns are hidden by default, so they must be explicitly selected) but unless the field is hooked in using the - * forBooleanValue/forLongValue/forBytesValue methods and the resulting FieldValueProvider is then passed into the appropriate row decoder, the fields - * will be null. Most values are assigned in the {@link com.facebook.presto.kafka.KafkaRecordSet}. - */ -public enum KafkaInternalFieldDescription -{ - /** - * _partition_id - Kafka partition id. - */ - PARTITION_ID_FIELD("_partition_id", BigintType.BIGINT, "Partition Id"), - - /** - * _partition_offset - The current offset of the message in the partition. - */ - PARTITION_OFFSET_FIELD("_partition_offset", BigintType.BIGINT, "Offset for the message within the partition"), - - /** - * _segment_start - Kafka start offset for the segment which contains the current message. This is per-partition. - */ - SEGMENT_START_FIELD("_segment_start", BigintType.BIGINT, "Segment start offset"), - - /** - * _segment_end - Kafka end offset for the segment which contains the current message. This is per-partition. The end offset is the first offset that is *not* in the segment. - */ - SEGMENT_END_FIELD("_segment_end", BigintType.BIGINT, "Segment end offset"), - - /** - * _segment_count - Running count of messages in a segment. - */ - SEGMENT_COUNT_FIELD("_segment_count", BigintType.BIGINT, "Running message count per segment"), - - /** - * _message_corrupt - True if the row converter could not read the a message. May be null if the row converter does not set a value (e.g. the dummy row converter does not). - */ - MESSAGE_CORRUPT_FIELD("_message_corrupt", BooleanType.BOOLEAN, "Message data is corrupt"), - - /** - * _message - Represents the full topic as a text column. Format is UTF-8 which may be wrong for some topics. TODO: make charset configurable. - */ - MESSAGE_FIELD("_message", createUnboundedVarcharType(), "Message text"), - - /** - * _message_length - length in bytes of the message. - */ - MESSAGE_LENGTH_FIELD("_message_length", BigintType.BIGINT, "Total number of message bytes"), - - /** - * _key_corrupt - True if the row converter could not read the a key. May be null if the row converter does not set a value (e.g. the dummy row converter does not). - */ - KEY_CORRUPT_FIELD("_key_corrupt", BooleanType.BOOLEAN, "Key data is corrupt"), - - /** - * _key - Represents the key as a text column. Format is UTF-8 which may be wrong for topics. TODO: make charset configurable. - */ - KEY_FIELD("_key", createUnboundedVarcharType(), "Key text"), - - /** - * _key_length - length in bytes of the key. - */ - KEY_LENGTH_FIELD("_key_length", BigintType.BIGINT, "Total number of key bytes"), - - /** - * _timestamp - offset timestamp, used to narrow scan range - */ - OFFSET_TIMESTAMP_FIELD("_timestamp", BigintType.BIGINT, "Offset Timestamp"); - - private static final Map BY_COLUMN_NAME = - stream(KafkaInternalFieldDescription.values()) - .collect(toImmutableMap(KafkaInternalFieldDescription::getColumnName, identity())); - - public static KafkaInternalFieldDescription forColumnName(String columnName) - { - KafkaInternalFieldDescription description = BY_COLUMN_NAME.get(columnName); - checkArgument(description != null, "Unknown internal column name %s", columnName); - return description; - } - - private final String columnName; - private final Type type; - private final String comment; - - KafkaInternalFieldDescription( - String columnName, - Type type, - String comment) - { - checkArgument(!isNullOrEmpty(columnName), "name is null or is empty"); - this.columnName = columnName; - this.type = requireNonNull(type, "type is null"); - this.comment = requireNonNull(comment, "comment is null"); - } - - public String getColumnName() - { - return columnName; - } - - public Type getType() - { - return type; - } - - KafkaColumnHandle getColumnHandle(String connectorId, int index, boolean hidden) - { - return new KafkaColumnHandle(connectorId, - index, - getColumnName(), - getType(), - null, - null, - null, - false, - hidden, - true); - } - - ColumnMetadata getColumnMetadata(boolean hidden) - { - return new ColumnMetadata(columnName, type, comment, hidden); - } -} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaMetadata.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaMetadata.java index 225c619c28ae..e69de29bb2d1 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaMetadata.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaMetadata.java @@ -1,274 +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.kafka; - -import com.facebook.presto.decoder.dummy.DummyRowDecoder; -import com.facebook.presto.spi.ColumnHandle; -import com.facebook.presto.spi.ColumnMetadata; -import com.facebook.presto.spi.ConnectorSession; -import com.facebook.presto.spi.ConnectorTableHandle; -import com.facebook.presto.spi.ConnectorTableLayout; -import com.facebook.presto.spi.ConnectorTableLayoutHandle; -import com.facebook.presto.spi.ConnectorTableLayoutResult; -import com.facebook.presto.spi.ConnectorTableMetadata; -import com.facebook.presto.spi.Constraint; -import com.facebook.presto.spi.SchemaTableName; -import com.facebook.presto.spi.SchemaTablePrefix; -import com.facebook.presto.spi.TableNotFoundException; -import com.facebook.presto.spi.connector.ConnectorMetadata; -import com.facebook.presto.spi.predicate.Domain; -import com.facebook.presto.spi.predicate.Marker; -import com.facebook.presto.spi.predicate.Range; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import io.airlift.log.Logger; - -import javax.inject.Inject; - -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.function.Supplier; - -import static com.facebook.presto.kafka.KafkaHandleResolver.convertColumnHandle; -import static com.facebook.presto.kafka.KafkaHandleResolver.convertTableHandle; -import static java.util.Objects.requireNonNull; - -/** - * Manages the Kafka connector specific metadata information. The Connector provides an additional set of columns - * for each table that are created as hidden columns. See {@link KafkaInternalFieldDescription} for a list - * of per-topic additional columns. - */ -public class KafkaMetadata - implements ConnectorMetadata -{ - private static final Logger log = Logger.get(KafkaMetadata.class); - - private final String connectorId; - private final boolean hideInternalColumns; - private final Map tableDescriptions; - private final Set internalFieldDescriptions; - - @Inject - public KafkaMetadata( - KafkaConnectorId connectorId, - KafkaConnectorConfig kafkaConnectorConfig, - Supplier> kafkaTableDescriptionSupplier, - Set internalFieldDescriptions) - { - this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); - - requireNonNull(kafkaConnectorConfig, "kafkaConfig is null"); - this.hideInternalColumns = kafkaConnectorConfig.isHideInternalColumns(); - - requireNonNull(kafkaTableDescriptionSupplier, "kafkaTableDescriptionSupplier is null"); - this.tableDescriptions = kafkaTableDescriptionSupplier.get(); - this.internalFieldDescriptions = requireNonNull(internalFieldDescriptions, "internalFieldDescriptions is null"); - } - - @Override - public List listSchemaNames(ConnectorSession session) - { - ImmutableSet.Builder builder = ImmutableSet.builder(); - for (SchemaTableName tableName : tableDescriptions.keySet()) { - builder.add(tableName.getSchemaName()); - } - return ImmutableList.copyOf(builder.build()); - } - - @Override - public KafkaTableHandle getTableHandle(ConnectorSession session, SchemaTableName schemaTableName) - { - KafkaTopicDescription table = tableDescriptions.get(schemaTableName); - if (table == null) { - return null; - } - - return new KafkaTableHandle(connectorId, - schemaTableName.getSchemaName(), - schemaTableName.getTableName(), - table.getTopicName(), - getDataFormat(table.getKey()), - getDataFormat(table.getMessage())); - } - - private static String getDataFormat(KafkaTopicFieldGroup fieldGroup) - { - return (fieldGroup == null) ? DummyRowDecoder.NAME : fieldGroup.getDataFormat(); - } - - @Override - public ConnectorTableMetadata getTableMetadata(ConnectorSession session, ConnectorTableHandle tableHandle) - { - return getTableMetadata(convertTableHandle(tableHandle).toSchemaTableName()); - } - - @Override - public List listTables(ConnectorSession session, String schemaNameOrNull) - { - ImmutableList.Builder builder = ImmutableList.builder(); - for (SchemaTableName tableName : tableDescriptions.keySet()) { - if (schemaNameOrNull == null || tableName.getSchemaName().equals(schemaNameOrNull)) { - builder.add(tableName); - } - } - - return builder.build(); - } - - @SuppressWarnings("ValueOfIncrementOrDecrementUsed") - @Override - public Map getColumnHandles(ConnectorSession session, ConnectorTableHandle tableHandle) - { - KafkaTableHandle kafkaTableHandle = convertTableHandle(tableHandle); - - KafkaTopicDescription kafkaTopicDescription = tableDescriptions.get(kafkaTableHandle.toSchemaTableName()); - if (kafkaTopicDescription == null) { - throw new TableNotFoundException(kafkaTableHandle.toSchemaTableName()); - } - - ImmutableMap.Builder columnHandles = ImmutableMap.builder(); - - int index = 0; - KafkaTopicFieldGroup key = kafkaTopicDescription.getKey(); - if (key != null) { - List fields = key.getFields(); - if (fields != null) { - for (KafkaTopicFieldDescription kafkaTopicFieldDescription : fields) { - columnHandles.put(kafkaTopicFieldDescription.getName(), kafkaTopicFieldDescription.getColumnHandle(connectorId, true, index++)); - } - } - } - - KafkaTopicFieldGroup message = kafkaTopicDescription.getMessage(); - if (message != null) { - List fields = message.getFields(); - if (fields != null) { - for (KafkaTopicFieldDescription kafkaTopicFieldDescription : fields) { - columnHandles.put(kafkaTopicFieldDescription.getName(), kafkaTopicFieldDescription.getColumnHandle(connectorId, false, index++)); - } - } - } - - for (KafkaInternalFieldDescription kafkaInternalFieldDescription : internalFieldDescriptions) { - columnHandles.put(kafkaInternalFieldDescription.getColumnName(), kafkaInternalFieldDescription.getColumnHandle(connectorId, index++, hideInternalColumns)); - } - - return columnHandles.build(); - } - - @Override - public Map> listTableColumns(ConnectorSession session, SchemaTablePrefix prefix) - { - requireNonNull(prefix, "prefix is null"); - - ImmutableMap.Builder> columns = ImmutableMap.builder(); - - List tableNames = prefix.getSchemaName() == null ? listTables(session, Optional.empty()) : ImmutableList.of(new SchemaTableName(prefix.getSchemaName(), prefix.getTableName())); - - for (SchemaTableName tableName : tableNames) { - ConnectorTableMetadata tableMetadata = getTableMetadata(tableName); - // table can disappear during listing operation - if (tableMetadata != null) { - columns.put(tableName, tableMetadata.getColumns()); - } - } - return columns.build(); - } - - @Override - public ColumnMetadata getColumnMetadata(ConnectorSession session, ConnectorTableHandle tableHandle, ColumnHandle columnHandle) - { - convertTableHandle(tableHandle); - return convertColumnHandle(columnHandle).getColumnMetadata(); - } - - @Override - public List getTableLayouts(ConnectorSession session, ConnectorTableHandle table, Constraint constraint, Optional> desiredColumns) - { - KafkaTableHandle handle = convertTableHandle(table); - - Long startTs = null; - Long endTs = null; - Optional> domains = constraint.getSummary().getDomains(); - log.info(constraint.getSummary().toString(null)); - if (domains.isPresent()) { - Map columnHandleDomainMap = domains.get(); - for (Map.Entry entry : columnHandleDomainMap.entrySet()) { - if (entry.getKey() instanceof KafkaColumnHandle && ((KafkaColumnHandle) entry.getKey()).getName().equals(KafkaInternalFieldDescription.OFFSET_TIMESTAMP_FIELD.getColumnName())) { - Range span = entry.getValue().getValues().getRanges().getSpan(); - Marker low = span.getLow(); - Marker high = span.getHigh(); - if (!low.isLowerUnbounded()) { - startTs = (Long) low.getValue(); - } - if (!high.isUpperUnbounded()) { - endTs = (Long) high.getValue(); - } - } - - log.info("K: %s\tV: %s", entry.getKey().toString(), entry.getValue().toString()); - } - } - - log.info("startTs: %s, endTs: %s", startTs, endTs); - ConnectorTableLayout layout = new ConnectorTableLayout(new KafkaTableLayoutHandle(handle, startTs, endTs)); - return ImmutableList.of(new ConnectorTableLayoutResult(layout, constraint.getSummary())); - } - - @Override - public ConnectorTableLayout getTableLayout(ConnectorSession session, ConnectorTableLayoutHandle handle) - { - return new ConnectorTableLayout(handle); - } - - @SuppressWarnings("ValueOfIncrementOrDecrementUsed") - private ConnectorTableMetadata getTableMetadata(SchemaTableName schemaTableName) - { - KafkaTopicDescription table = tableDescriptions.get(schemaTableName); - if (table == null) { - throw new TableNotFoundException(schemaTableName); - } - - ImmutableList.Builder builder = ImmutableList.builder(); - - KafkaTopicFieldGroup key = table.getKey(); - if (key != null) { - List fields = key.getFields(); - if (fields != null) { - for (KafkaTopicFieldDescription fieldDescription : fields) { - builder.add(fieldDescription.getColumnMetadata()); - } - } - } - - KafkaTopicFieldGroup message = table.getMessage(); - if (message != null) { - List fields = message.getFields(); - if (fields != null) { - for (KafkaTopicFieldDescription fieldDescription : fields) { - builder.add(fieldDescription.getColumnMetadata()); - } - } - } - - for (KafkaInternalFieldDescription fieldDescription : internalFieldDescriptions) { - builder.add(fieldDescription.getColumnMetadata(hideInternalColumns)); - } - - return new ConnectorTableMetadata(schemaTableName, builder.build()); - } -} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaPlugin.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaPlugin.java deleted file mode 100644 index 284424a9a7fa..000000000000 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaPlugin.java +++ /dev/null @@ -1,47 +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.kafka; - -import com.facebook.presto.spi.Plugin; -import com.facebook.presto.spi.SchemaTableName; -import com.facebook.presto.spi.connector.ConnectorFactory; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableList; - -import java.util.Map; -import java.util.Optional; -import java.util.function.Supplier; - -import static java.util.Objects.requireNonNull; - -/** - * Presto plugin to use Apache Kafka as a data source. - */ -public class KafkaPlugin - implements Plugin -{ - private Optional>> tableDescriptionSupplier = Optional.empty(); - - @VisibleForTesting - public synchronized void setTableDescriptionSupplier(Supplier> tableDescriptionSupplier) - { - this.tableDescriptionSupplier = Optional.of(requireNonNull(tableDescriptionSupplier, "tableDescriptionSupplier is null")); - } - - @Override - public synchronized Iterable getConnectorFactories() - { - return ImmutableList.of(new KafkaConnectorFactory(tableDescriptionSupplier)); - } -} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java index 2827d8bd5609..e69de29bb2d1 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSet.java @@ -1,353 +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.kafka; - -import com.facebook.presto.decoder.DecoderColumnHandle; -import com.facebook.presto.decoder.FieldValueProvider; -import com.facebook.presto.decoder.RowDecoder; -import com.facebook.presto.spi.ColumnHandle; -import com.facebook.presto.spi.PrestoException; -import com.facebook.presto.spi.RecordCursor; -import com.facebook.presto.spi.RecordSet; -import com.facebook.presto.spi.block.Block; -import com.facebook.presto.spi.type.Type; -import com.google.common.collect.ImmutableList; -import io.airlift.log.Logger; -import io.airlift.slice.Slice; -import kafka.api.FetchRequest; -import kafka.api.OffsetRequest; -import kafka.common.ErrorMapping; -import kafka.common.OffsetOutOfRangeException; -import kafka.javaapi.consumer.SimpleConsumer; -import kafka.javaapi.message.ByteBufferMessageSet; -import kafka.message.MessageAndOffset; - -import java.nio.ByteBuffer; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.atomic.AtomicBoolean; - -import static com.facebook.presto.decoder.FieldValueProviders.booleanValueProvider; -import static com.facebook.presto.decoder.FieldValueProviders.bytesValueProvider; -import static com.facebook.presto.decoder.FieldValueProviders.longValueProvider; -import static com.facebook.presto.kafka.KafkaErrorCode.KAFKA_SPLIT_ERROR; -import static com.google.common.base.Preconditions.checkArgument; -import static java.util.Objects.requireNonNull; - -/** - * Kafka specific record set. Returns a cursor for a topic which iterates over a Kafka partition segment. - */ -public class KafkaRecordSet - implements RecordSet -{ - private static final Logger log = Logger.get(KafkaRecordSet.class); - - private static final byte[] EMPTY_BYTE_ARRAY = new byte[0]; - - private final KafkaSplit split; - private final KafkaSimpleConsumerManager consumerManager; - private final int fetchSize; - - private final RowDecoder keyDecoder; - private final RowDecoder messageDecoder; - - private final List columnHandles; - private final List columnTypes; - - KafkaRecordSet(KafkaSplit split, - KafkaSimpleConsumerManager consumerManager, - List columnHandles, - RowDecoder keyDecoder, - RowDecoder messageDecoder, - int fetchSize) - { - this.split = requireNonNull(split, "split is null"); - - this.consumerManager = requireNonNull(consumerManager, "consumerManager is null"); - - this.keyDecoder = requireNonNull(keyDecoder, "rowDecoder is null"); - this.messageDecoder = requireNonNull(messageDecoder, "rowDecoder is null"); - - this.columnHandles = requireNonNull(columnHandles, "columnHandles is null"); - - ImmutableList.Builder typeBuilder = ImmutableList.builder(); - - for (DecoderColumnHandle handle : columnHandles) { - typeBuilder.add(handle.getType()); - } - - this.columnTypes = typeBuilder.build(); - this.fetchSize = fetchSize; - } - - @Override - public List getColumnTypes() - { - return columnTypes; - } - - @Override - public RecordCursor cursor() - { - return new KafkaRecordCursor(split.getStartTs(), split.getEndTs()); - } - - public class KafkaRecordCursor - implements RecordCursor - { - private long totalBytes; - private long totalMessages; - private long cursorOffset = split.getStart(); - private Iterator messageAndOffsetIterator; - private long fetchedSize; - private final AtomicBoolean reported = new AtomicBoolean(); - - private final long startTs; - private final long endTs; - - private FieldValueProvider[] currentRowValues = new FieldValueProvider[columnHandles.size()]; - - KafkaRecordCursor(long startTs, long endTs) - { - this.startTs = startTs; - this.endTs = endTs; - } - - @Override - public long getCompletedBytes() - { - return totalBytes; - } - - @Override - public long getReadTimeNanos() - { - return 0; - } - - @Override - public Type getType(int field) - { - checkArgument(field < columnHandles.size(), "Invalid field index"); - return columnHandles.get(field).getType(); - } - - @Override - public boolean advanceNextPosition() - { - while (true) { - if (cursorOffset >= split.getEnd()) { - return endOfData(1); // Split end is exclusive. - } - - try { - // Create a fetch request - openFetchRequest(); - if (cursorOffset >= split.getEnd()) { - return endOfData(2); // Split end is exclusive. - } - if (messageAndOffsetIterator.hasNext()) { - MessageAndOffset currentMessageAndOffset = messageAndOffsetIterator.next(); - return nextRow(currentMessageAndOffset); - } - } - catch (OffsetOutOfRangeException e) { - e.printStackTrace(); - return endOfData(4); - } - messageAndOffsetIterator = null; - } - } - - private boolean endOfData(int from) - { - if (!reported.getAndSet(true)) { - log.info("Found (from %d) a total of %d messages with %d bytes (%d compressed bytes expected). Last Offset: %d (%d, %d)", - from, totalMessages, totalBytes, split.getEnd() - split.getStart(), - cursorOffset, split.getStart(), split.getEnd()); - } - return false; - } - - private boolean nextRow(MessageAndOffset messageAndOffset) - { - totalBytes += messageAndOffset.message().payloadSize(); - totalMessages++; - - byte[] keyData = EMPTY_BYTE_ARRAY; - byte[] messageData = EMPTY_BYTE_ARRAY; - - ByteBuffer message = messageAndOffset.message().payload(); - if (message != null) { - messageData = new byte[message.remaining()]; - message.get(messageData); - } - - Map currentRowValuesMap = new HashMap<>(); - - Optional> decodedKey = keyDecoder.decodeRow(keyData, null); - Optional> decodedValue = messageDecoder.decodeRow(messageData, null); - - for (DecoderColumnHandle columnHandle : columnHandles) { - if (columnHandle.isInternal()) { - KafkaInternalFieldDescription fieldDescription = KafkaInternalFieldDescription.forColumnName(columnHandle.getName()); - switch (fieldDescription) { - case SEGMENT_COUNT_FIELD: - currentRowValuesMap.put(columnHandle, longValueProvider(totalMessages)); - break; - case PARTITION_OFFSET_FIELD: - currentRowValuesMap.put(columnHandle, longValueProvider(messageAndOffset.offset())); - break; - case MESSAGE_FIELD: - currentRowValuesMap.put(columnHandle, bytesValueProvider(messageData)); - break; - case MESSAGE_LENGTH_FIELD: - currentRowValuesMap.put(columnHandle, longValueProvider(messageData.length)); - break; - case KEY_FIELD: - currentRowValuesMap.put(columnHandle, bytesValueProvider(keyData)); - break; - case KEY_LENGTH_FIELD: - currentRowValuesMap.put(columnHandle, longValueProvider(keyData.length)); - break; - case KEY_CORRUPT_FIELD: - currentRowValuesMap.put(columnHandle, booleanValueProvider(!decodedKey.isPresent())); - break; - case MESSAGE_CORRUPT_FIELD: - currentRowValuesMap.put(columnHandle, booleanValueProvider(!decodedValue.isPresent())); - break; - case PARTITION_ID_FIELD: - currentRowValuesMap.put(columnHandle, longValueProvider(split.getPartitionId())); - break; - case SEGMENT_START_FIELD: - currentRowValuesMap.put(columnHandle, longValueProvider(startTs)); - break; - case SEGMENT_END_FIELD: - currentRowValuesMap.put(columnHandle, longValueProvider(endTs)); - break; - case OFFSET_TIMESTAMP_FIELD: - currentRowValuesMap.put(columnHandle, longValueProvider(populateOffsetTimestamp(startTs, endTs))); - default: - throw new IllegalArgumentException("unknown internal field " + fieldDescription); - } - } - } - - decodedKey.ifPresent(currentRowValuesMap::putAll); - decodedValue.ifPresent(currentRowValuesMap::putAll); - - for (int i = 0; i < columnHandles.size(); i++) { - ColumnHandle columnHandle = columnHandles.get(i); - currentRowValues[i] = currentRowValuesMap.get(columnHandle); - } - - return true; // Advanced successfully. - } - - private void openFetchRequest() - { - if (messageAndOffsetIterator == null) { - log.info("Fetching %d bytes from partition %d @offset %d (%d - %d) -- %d messages read so far", - fetchSize, split.getPartitionId(), cursorOffset, split.getStart(), split.getEnd(), totalMessages); - cursorOffset += fetchedSize; - if (cursorOffset < split.getEnd()) { - FetchRequest req = new FetchRequest(split.getTopicName(), split.getPartitionId(), cursorOffset, fetchSize); - SimpleConsumer consumer = consumerManager.getConsumer(split.getLeader()); - - ByteBufferMessageSet fetch = consumer.fetch(req); - log.debug("\t...fetched %s bytes, validBytes=%s, initialOffset=%s", fetch.sizeInBytes(), fetch.validBytes(), fetch.getInitialOffset()); - int errorCode = fetch.getErrorCode(); - if (errorCode != ErrorMapping.NoError() && errorCode != ErrorMapping.OffsetOutOfRangeCode()) { - log.warn("Fetch response has error: %d", errorCode); - throw new PrestoException(KAFKA_SPLIT_ERROR, "could not fetch data from Kafka, error code is '" + errorCode + "'"); - } - - fetchedSize = fetch.validBytes(); - messageAndOffsetIterator = fetch.iterator(); - } - } - } - - private long populateOffsetTimestamp(long startTs, long endTs) - { - if (startTs == OffsetRequest.EarliestTime()) { - startTs = 0; - } - - if (endTs == OffsetRequest.LatestTime()) { - endTs = System.currentTimeMillis(); - } - - return startTs + (endTs - startTs) / 2; - } - - @Override - public boolean getBoolean(int field) - { - return getFieldValueProvider(field, boolean.class).getBoolean(); - } - - @Override - public long getLong(int field) - { - return getFieldValueProvider(field, long.class).getLong(); - } - - @Override - public double getDouble(int field) - { - return getFieldValueProvider(field, double.class).getDouble(); - } - - @Override - public Slice getSlice(int field) - { - return getFieldValueProvider(field, Slice.class).getSlice(); - } - - @Override - public Object getObject(int field) - { - return getFieldValueProvider(field, Block.class).getBlock(); - } - - @Override - public boolean isNull(int field) - { - checkArgument(field < columnHandles.size(), "Invalid field index"); - return currentRowValues[field] == null || currentRowValues[field].isNull(); - } - - private FieldValueProvider getFieldValueProvider(int field, Class expectedType) - { - checkArgument(field < columnHandles.size(), "Invalid field index"); - checkFieldType(field, expectedType); - return currentRowValues[field]; - } - - private void checkFieldType(int field, Class expected) - { - Class actual = getType(field).getJavaType(); - checkArgument(actual == expected, "Expected field %s to be type %s but is %s", field, expected, actual); - } - - @Override - public void close() - { - } - } -} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSetProvider.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSetProvider.java index b926e1c44323..e69de29bb2d1 100644 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSetProvider.java +++ b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaRecordSetProvider.java @@ -1,80 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.facebook.presto.kafka; - -import com.facebook.presto.decoder.DispatchingRowDecoderFactory; -import com.facebook.presto.decoder.RowDecoder; -import com.facebook.presto.spi.ColumnHandle; -import com.facebook.presto.spi.ConnectorSession; -import com.facebook.presto.spi.ConnectorSplit; -import com.facebook.presto.spi.RecordSet; -import com.facebook.presto.spi.connector.ConnectorRecordSetProvider; -import com.facebook.presto.spi.connector.ConnectorTransactionHandle; -import com.google.common.collect.ImmutableList; - -import javax.inject.Inject; - -import java.util.List; - -import static com.facebook.presto.kafka.KafkaHandleResolver.convertSplit; -import static com.google.common.collect.ImmutableSet.toImmutableSet; -import static java.util.Collections.emptyMap; -import static java.util.Objects.requireNonNull; - -/** - * Factory for Kafka specific {@link RecordSet} instances. - */ -public class KafkaRecordSetProvider - implements ConnectorRecordSetProvider -{ - private final KafkaSimpleConsumerManager consumerManager; - private final DispatchingRowDecoderFactory decoderFactory; - private final KafkaConnectorConfig config; - - @Inject - public KafkaRecordSetProvider(DispatchingRowDecoderFactory decoderFactory, KafkaSimpleConsumerManager consumerManager, KafkaConnectorConfig config) - { - this.decoderFactory = requireNonNull(decoderFactory, "decoderFactory is null"); - this.consumerManager = requireNonNull(consumerManager, "consumerManager is null"); - this.config = requireNonNull(config, "config is null"); - } - - @Override - public RecordSet getRecordSet(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorSplit split, List columns) - { - KafkaSplit kafkaSplit = convertSplit(split); - - List kafkaColumns = columns.stream() - .map(KafkaHandleResolver::convertColumnHandle) - .collect(ImmutableList.toImmutableList()); - - RowDecoder keyDecoder = decoderFactory.create( - kafkaSplit.getKeyDataFormat(), - emptyMap(), - kafkaColumns.stream() - .filter(col -> !col.isInternal()) - .filter(KafkaColumnHandle::isKeyDecoder) - .collect(toImmutableSet())); - - RowDecoder messageDecoder = decoderFactory.create( - kafkaSplit.getMessageDataFormat(), - emptyMap(), - kafkaColumns.stream() - .filter(col -> !col.isInternal()) - .filter(col -> !col.isKeyDecoder()) - .collect(toImmutableSet())); - - return new KafkaRecordSet(kafkaSplit, consumerManager, kafkaColumns, keyDecoder, messageDecoder, config.getFetchSize()); - } -} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSimpleConsumerManager.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSimpleConsumerManager.java deleted file mode 100644 index 53756c614efa..000000000000 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSimpleConsumerManager.java +++ /dev/null @@ -1,103 +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.kafka; - -import com.facebook.presto.spi.HostAddress; -import com.facebook.presto.spi.NodeManager; -import com.google.common.base.Throwables; -import com.google.common.cache.CacheBuilder; -import com.google.common.cache.CacheLoader; -import com.google.common.cache.LoadingCache; -import io.airlift.log.Logger; -import kafka.javaapi.consumer.SimpleConsumer; - -import javax.annotation.PreDestroy; -import javax.inject.Inject; - -import java.util.Map; -import java.util.concurrent.ExecutionException; - -import static java.lang.Math.toIntExact; -import static java.util.Objects.requireNonNull; - -/** - * Manages connections to the Kafka nodes. A worker may connect to multiple Kafka nodes depending on the segments and partitions - * it needs to process. According to the Kafka source code, a Kafka {@link kafka.javaapi.consumer.SimpleConsumer} is thread-safe. - */ -public class KafkaSimpleConsumerManager -{ - private static final Logger log = Logger.get(KafkaSimpleConsumerManager.class); - - private final LoadingCache consumerCache; - - private final String connectorId; - private final NodeManager nodeManager; - private final int connectTimeoutMillis; - private final int bufferSizeBytes; - - @Inject - public KafkaSimpleConsumerManager( - KafkaConnectorId connectorId, - KafkaConnectorConfig kafkaConnectorConfig, - NodeManager nodeManager) - { - this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); - this.nodeManager = requireNonNull(nodeManager, "nodeManager is null"); - - requireNonNull(kafkaConnectorConfig, "kafkaConfig is null"); - this.connectTimeoutMillis = toIntExact(kafkaConnectorConfig.getKafkaConnectTimeout().toMillis()); - this.bufferSizeBytes = toIntExact(kafkaConnectorConfig.getKafkaBufferSize().toBytes()); - - this.consumerCache = CacheBuilder.newBuilder().build(new SimpleConsumerCacheLoader()); - } - - @PreDestroy - public void tearDown() - { - for (Map.Entry entry : consumerCache.asMap().entrySet()) { - try { - entry.getValue().close(); - } - catch (Exception e) { - log.warn(e, "While closing consumer %s:", entry.getKey()); - } - } - } - - public SimpleConsumer getConsumer(HostAddress host) - { - requireNonNull(host, "host is null"); - try { - return consumerCache.get(host); - } - catch (ExecutionException e) { - throw Throwables.propagate(e.getCause()); - } - } - - private class SimpleConsumerCacheLoader - extends CacheLoader - { - @Override - public SimpleConsumer load(HostAddress host) - throws Exception - { - log.debug("Creating new Consumer for %s", host); - return new SimpleConsumer(host.getHostText(), - host.getPort(), - connectTimeoutMillis, - bufferSizeBytes); - } - } -} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplit.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplit.java deleted file mode 100644 index 17edf7d47e6e..000000000000 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplit.java +++ /dev/null @@ -1,165 +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.kafka; - -import com.facebook.presto.spi.ConnectorSplit; -import com.facebook.presto.spi.HostAddress; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.ImmutableList; - -import java.util.List; - -import static com.google.common.base.MoreObjects.toStringHelper; -import static java.util.Objects.requireNonNull; - -/** - * Represents a kafka specific {@link ConnectorSplit}. Each split is mapped to a segment file on disk (based off the segment offset start() and end() values) so that - * a partition can be processed by reading segment files from partition leader. Otherwise, a Kafka topic could only be processed along partition boundaries. - *

- * When planning to process a Kafka topic with Presto, using smaller than the recommended segment size (default is 1G) allows Presto to optimize early and process a topic - * with more workers in parallel. - */ -public class KafkaSplit - implements ConnectorSplit -{ - private final String connectorId; - private final String topicName; - private final String keyDataFormat; - private final String messageDataFormat; - private final int partitionId; - private final long start; - private final long end; - private final HostAddress leader; - private final long startTs; - private final long endTs; - - @JsonCreator - public KafkaSplit( - @JsonProperty("connectorId") String connectorId, - @JsonProperty("topicName") String topicName, - @JsonProperty("keyDataFormat") String keyDataFormat, - @JsonProperty("messageDataFormat") String messageDataFormat, - @JsonProperty("partitionId") int partitionId, - @JsonProperty("start") long start, - @JsonProperty("end") long end, - @JsonProperty("leader") HostAddress leader, - @JsonProperty("startTs") long startTs, - @JsonProperty("endTs") long endTs) - { - this.connectorId = requireNonNull(connectorId, "connector id is null"); - this.topicName = requireNonNull(topicName, "topicName is null"); - this.keyDataFormat = requireNonNull(keyDataFormat, "dataFormat is null"); - this.messageDataFormat = requireNonNull(messageDataFormat, "messageDataFormat is null"); - this.partitionId = partitionId; - this.start = start; - this.end = end; - this.leader = requireNonNull(leader, "leader address is null"); - this.startTs = startTs; - this.endTs = endTs; - } - - @JsonProperty - public String getConnectorId() - { - return connectorId; - } - - @JsonProperty - public long getStart() - { - return start; - } - - @JsonProperty - public long getEnd() - { - return end; - } - - @JsonProperty - public String getTopicName() - { - return topicName; - } - - @JsonProperty - public String getKeyDataFormat() - { - return keyDataFormat; - } - - @JsonProperty - public String getMessageDataFormat() - { - return messageDataFormat; - } - - @JsonProperty - public int getPartitionId() - { - return partitionId; - } - - @JsonProperty - public HostAddress getLeader() - { - return leader; - } - - @JsonProperty - public long getStartTs() - { - return startTs; - } - - @JsonProperty - public long getEndTs() - { - return endTs; - } - - @Override - public boolean isRemotelyAccessible() - { - return true; - } - - @Override - public List getAddresses() - { - return ImmutableList.of(leader); - } - - @Override - public Object getInfo() - { - return this; - } - - @Override - public String toString() - { - return toStringHelper(this) - .add("connectorId", connectorId) - .add("topicName", topicName) - .add("keyDataFormat", keyDataFormat) - .add("messageDataFormat", messageDataFormat) - .add("partitionId", partitionId) - .add("start", start) - .add("end", end) - .add("leader", leader) - .toString(); - } -} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java deleted file mode 100644 index 056bfea7e2b2..000000000000 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaSplitManager.java +++ /dev/null @@ -1,153 +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.kafka; - -import com.facebook.presto.spi.ConnectorSession; -import com.facebook.presto.spi.ConnectorSplit; -import com.facebook.presto.spi.ConnectorSplitSource; -import com.facebook.presto.spi.ConnectorTableLayoutHandle; -import com.facebook.presto.spi.FixedSplitSource; -import com.facebook.presto.spi.HostAddress; -import com.facebook.presto.spi.connector.ConnectorSplitManager; -import com.facebook.presto.spi.connector.ConnectorTransactionHandle; -import com.google.common.collect.ImmutableList; -import io.airlift.log.Logger; -import kafka.api.OffsetRequest; -import kafka.cluster.Broker; -import kafka.cluster.Cluster; -import kafka.cluster.Partition; -import kafka.javaapi.consumer.SimpleConsumer; -import kafka.utils.ZkUtils; -import org.I0Itec.zkclient.ZkClient; - -import javax.inject.Inject; - -import java.util.List; - -import static com.facebook.presto.kafka.KafkaHandleResolver.convertLayout; -import static java.util.Objects.requireNonNull; - -/** - * Kafka specific implementation of {@link ConnectorSplitManager}. - */ -public class KafkaSplitManager - implements ConnectorSplitManager -{ - private static final Logger log = Logger.get(KafkaSplitManager.class); - - private final String connectorId; - private final KafkaSimpleConsumerManager consumerManager; - private final KafkaConnectorConfig config; - - @Inject - public KafkaSplitManager( - KafkaConnectorId connectorId, - KafkaConnectorConfig kafkaConnectorConfig, - KafkaSimpleConsumerManager consumerManager) - { - this.connectorId = requireNonNull(connectorId, "connectorId is null").toString(); - this.consumerManager = requireNonNull(consumerManager, "consumerManager is null"); - - requireNonNull(kafkaConnectorConfig, "kafkaConfig is null"); - this.config = kafkaConnectorConfig; - } - - @Override - public ConnectorSplitSource getSplits(ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableLayoutHandle layout, SplitSchedulingStrategy splitSchedulingStrategy) - { - KafkaTableHandle kafkaTableHandle = convertLayout(layout).getTable(); - ZkClient zkClient = KafkaUtil.newZkClient(config.getZkEndpoint()); - - ImmutableList.Builder splits = ImmutableList.builder(); - Cluster cluster = ZkUtils.getCluster(zkClient); - List partitions = KafkaUtil.getPartitionsForTopic(zkClient, kafkaTableHandle.getTopicName()); - - long estimatedTotalSize = 0L; - - for (Partition part : partitions) { - log.debug("Adding Partition %s/%s from broker %s", kafkaTableHandle.getTopicName(), part.partId(), part.brokerId()); - Broker leader = cluster.getBroker(part.brokerId()).get(); - - if (leader == null) { // Leader election going on... - log.error("No leader for partition %s/%s found!", kafkaTableHandle.getTopicName(), part.partId()); - continue; - } - - HostAddress partitionLeader = HostAddress.fromParts(leader.host(), leader.port()); - - SimpleConsumer leaderConsumer = consumerManager.getConsumer(partitionLeader); - // Kafka contains a reverse list of "end - start" pairs for the splits - - KafkaTableLayoutHandle layoutHandle = (KafkaTableLayoutHandle) layout; - long startTs = layoutHandle.getOffsetStartTs(); - long endTs = layoutHandle.getOffsetEndTs(); - - long[] offsets = findAllOffsets(leaderConsumer, kafkaTableHandle.getTopicName(), part.partId(), startTs, endTs); - for (int i = offsets.length - 1; i > 0; i--) { - KafkaSplit split = new KafkaSplit( - connectorId, - kafkaTableHandle.getTopicName(), - kafkaTableHandle.getKeyDataFormat(), - kafkaTableHandle.getMessageDataFormat(), - part.partId(), - offsets[i], - offsets[i - 1], - partitionLeader, - startTs, - endTs); - splits.add(split); - - long splitSize = (split.getEnd() - split.getStart()) / 1024 / 1024; - log.debug("Split summarize: %s-%s (%sMB)", split.getStart(), split.getEnd(), splitSize); - estimatedTotalSize += splitSize; - } - } - - ImmutableList builtSplits = splits.build(); - log.info("Built " + builtSplits.size() + " splits"); - - log.info("EstimatedTotalSize: %s", estimatedTotalSize); - return new FixedSplitSource(builtSplits); - } - - private static long[] findAllOffsets(SimpleConsumer consumer, String topicName, int partitionId, long startTs, long endTs) - { - // startTs: start timestamp, or -2/null as earliest - // endTs: end timestamp, or -1/null as latest - if (startTs >= endTs && endTs != OffsetRequest.LatestTime()) { - throw new IllegalArgumentException(String.format("Invalid Kafka Offset start/end pair: %s - %s", startTs, endTs)); - } - - long[] offsetsBeforeStartTs = consumer.getOffsetsBefore(topicName, partitionId, startTs, Integer.MAX_VALUE); - long[] offsetsBeforeEndTs = consumer.getOffsetsBefore(topicName, partitionId, endTs, Integer.MAX_VALUE); - log.debug("NumOffsetsBeforeStartTs=%s, NumOffsetsBeforeEndTs=%s", offsetsBeforeStartTs.length, offsetsBeforeEndTs.length); - - if (offsetsBeforeStartTs.length == 0) { - return offsetsBeforeEndTs; - } - - long[] offsets = new long[offsetsBeforeEndTs.length - offsetsBeforeStartTs.length + 1]; - long startOffset = offsetsBeforeStartTs[0]; - - for (int i = 0; i < offsetsBeforeEndTs.length; i++) { - if (offsetsBeforeEndTs[i] == startOffset) { - offsets[i] = startOffset; - break; - } - offsets[i] = offsetsBeforeEndTs[i]; - } - - return offsets; - } -} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableDescriptionSupplier.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableDescriptionSupplier.java deleted file mode 100644 index 2fcdbe539ef4..000000000000 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableDescriptionSupplier.java +++ /dev/null @@ -1,138 +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.kafka; - -import com.facebook.presto.decoder.dummy.DummyRowDecoder; -import com.facebook.presto.spi.SchemaTableName; -import com.google.common.base.Splitter; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import io.airlift.json.JsonCodec; -import io.airlift.log.Logger; - -import javax.inject.Inject; - -import java.io.File; -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.function.Supplier; - -import static com.google.common.base.MoreObjects.firstNonNull; -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Strings.isNullOrEmpty; -import static java.nio.file.Files.readAllBytes; -import static java.util.Arrays.asList; -import static java.util.Objects.requireNonNull; - -public class KafkaTableDescriptionSupplier - implements Supplier> -{ - private static final Logger log = Logger.get(KafkaTableDescriptionSupplier.class); - - private final JsonCodec topicDescriptionCodec; - private final File tableDescriptionDir; - private final String defaultSchema; - private final Set tableNames; - - @Inject - KafkaTableDescriptionSupplier(KafkaConnectorConfig kafkaConnectorConfig, - JsonCodec topicDescriptionCodec) - { - this.topicDescriptionCodec = requireNonNull(topicDescriptionCodec, "topicDescriptionCodec is null"); - - requireNonNull(kafkaConnectorConfig, "kafkaConfig is null"); - this.tableDescriptionDir = kafkaConnectorConfig.getTableDescriptionDir(); - this.defaultSchema = kafkaConnectorConfig.getDefaultSchema(); - this.tableNames = ImmutableSet.copyOf(kafkaConnectorConfig.getTableNames()); - } - - @Override - public Map get() - { - ImmutableMap.Builder builder = ImmutableMap.builder(); - - log.debug("Loading kafka table definitions from %s", tableDescriptionDir.getAbsolutePath()); - - try { - for (File file : listFiles(tableDescriptionDir)) { - if (file.isFile() && file.getName().endsWith(".json")) { - KafkaTopicDescription table = topicDescriptionCodec.fromJson(readAllBytes(file.toPath())); - String schemaName = firstNonNull(table.getSchemaName(), defaultSchema); - log.debug("Kafka table %s.%s: %s", schemaName, table.getTableName(), table); - builder.put(new SchemaTableName(schemaName, table.getTableName()), table); - } - } - - Map tableDefinitions = builder.build(); - - log.debug("Loaded Table definitions: %s", tableDefinitions.keySet()); - - builder = ImmutableMap.builder(); - for (String definedTable : tableNames) { - SchemaTableName tableName; - try { - tableName = parseTableName(definedTable); - } - catch (IllegalArgumentException iae) { - tableName = new SchemaTableName(defaultSchema, definedTable); - } - - if (tableDefinitions.containsKey(tableName)) { - KafkaTopicDescription kafkaTable = tableDefinitions.get(tableName); - log.debug("Found Table definition for %s: %s", tableName, kafkaTable); - builder.put(tableName, kafkaTable); - } - else { - // A dummy table definition only supports the internal columns. - log.debug("Created dummy Table definition for %s", tableName); - builder.put(tableName, new KafkaTopicDescription(tableName.getTableName(), - tableName.getSchemaName(), - definedTable, - new KafkaTopicFieldGroup(DummyRowDecoder.NAME, ImmutableList.of()), - new KafkaTopicFieldGroup(DummyRowDecoder.NAME, ImmutableList.of()))); - } - } - - return builder.build(); - } - catch (IOException e) { - log.warn(e, "Error: "); - throw Throwables.propagate(e); - } - } - - private static List listFiles(File dir) - { - if ((dir != null) && dir.isDirectory()) { - File[] files = dir.listFiles(); - if (files != null) { - log.debug("Considering files: %s", asList(files)); - return ImmutableList.copyOf(files); - } - } - return ImmutableList.of(); - } - - private static SchemaTableName parseTableName(String schemaTableName) - { - checkArgument(!isNullOrEmpty(schemaTableName), "schemaTableName is null or is empty"); - List parts = Splitter.on('.').splitToList(schemaTableName); - checkArgument(parts.size() == 2, "Invalid schemaTableName: %s", schemaTableName); - return new SchemaTableName(parts.get(0), parts.get(1)); - } -} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableHandle.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableHandle.java deleted file mode 100644 index 8475b2c68c98..000000000000 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableHandle.java +++ /dev/null @@ -1,151 +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.kafka; - -import com.facebook.presto.spi.ConnectorTableHandle; -import com.facebook.presto.spi.SchemaTableName; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.util.Objects; - -import static com.google.common.base.MoreObjects.toStringHelper; -import static java.util.Objects.requireNonNull; - -/** - * Kafka specific {@link ConnectorTableHandle}. - */ -public final class KafkaTableHandle - implements ConnectorTableHandle -{ - /** - * connector id - */ - private final String connectorId; - - /** - * The schema name for this table. Is set through configuration and read - * using {@link KafkaConnectorConfig#getDefaultSchema()}. Usually 'default'. - */ - private final String schemaName; - - /** - * The table name used by presto. - */ - private final String tableName; - - /** - * The topic name that is read from Kafka. - */ - private final String topicName; - - private final String keyDataFormat; - private final String messageDataFormat; - - @JsonCreator - public KafkaTableHandle( - @JsonProperty("connectorId") String connectorId, - @JsonProperty("schemaName") String schemaName, - @JsonProperty("tableName") String tableName, - @JsonProperty("topicName") String topicName, - @JsonProperty("keyDataFormat") String keyDataFormat, - @JsonProperty("messageDataFormat") String messageDataFormat) - { - this.connectorId = requireNonNull(connectorId, "connectorId is null"); - this.schemaName = requireNonNull(schemaName, "schemaName is null"); - this.tableName = requireNonNull(tableName, "tableName is null"); - this.topicName = requireNonNull(topicName, "topicName is null"); - this.keyDataFormat = requireNonNull(keyDataFormat, "keyDataFormat is null"); - this.messageDataFormat = requireNonNull(messageDataFormat, "messageDataFormat is null"); - } - - @JsonProperty - public String getConnectorId() - { - return connectorId; - } - - @JsonProperty - public String getSchemaName() - { - return schemaName; - } - - @JsonProperty - public String getTableName() - { - return tableName; - } - - @JsonProperty - public String getTopicName() - { - return topicName; - } - - @JsonProperty - public String getKeyDataFormat() - { - return keyDataFormat; - } - - @JsonProperty - public String getMessageDataFormat() - { - return messageDataFormat; - } - - public SchemaTableName toSchemaTableName() - { - return new SchemaTableName(schemaName, tableName); - } - - @Override - public int hashCode() - { - return Objects.hash(connectorId, schemaName, tableName, topicName, keyDataFormat, messageDataFormat); - } - - @Override - public boolean equals(Object obj) - { - if (this == obj) { - return true; - } - if (obj == null || getClass() != obj.getClass()) { - return false; - } - - KafkaTableHandle other = (KafkaTableHandle) obj; - return Objects.equals(this.connectorId, other.connectorId) - && Objects.equals(this.schemaName, other.schemaName) - && Objects.equals(this.tableName, other.tableName) - && Objects.equals(this.topicName, other.topicName) - && Objects.equals(this.keyDataFormat, other.keyDataFormat) - && Objects.equals(this.messageDataFormat, other.messageDataFormat); - } - - @Override - public String toString() - { - return toStringHelper(this) - .add("connectorId", connectorId) - .add("schemaName", schemaName) - .add("tableName", tableName) - .add("topicName", topicName) - .add("keyDataFormat", keyDataFormat) - .add("messageDataFormat", messageDataFormat) - .toString(); - } -} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableLayoutHandle.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableLayoutHandle.java deleted file mode 100644 index 57e6ffc6e615..000000000000 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTableLayoutHandle.java +++ /dev/null @@ -1,64 +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.kafka; - -import com.facebook.presto.spi.ConnectorTableLayoutHandle; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import kafka.api.OffsetRequest; - -import static java.util.Objects.requireNonNull; - -public class KafkaTableLayoutHandle - implements ConnectorTableLayoutHandle -{ - private final KafkaTableHandle table; - private final long offsetStartTs; - private final long offsetEndTs; - - @JsonCreator - public KafkaTableLayoutHandle( - @JsonProperty("table") KafkaTableHandle table, - @JsonProperty("offset_start_ts") Long offsetStartTs, - @JsonProperty("offset_end_ts") Long offsetEndTs) - { - this.table = requireNonNull(table, "table is null"); - this.offsetStartTs = offsetStartTs == null ? OffsetRequest.EarliestTime() : offsetStartTs; - this.offsetEndTs = offsetEndTs == null ? OffsetRequest.LatestTime() : offsetEndTs; - } - - @JsonProperty - public KafkaTableHandle getTable() - { - return table; - } - - @JsonProperty - public long getOffsetStartTs() - { - return offsetStartTs; - } - - @JsonProperty - public long getOffsetEndTs() - { - return offsetEndTs; - } - - @Override - public String toString() - { - return table.toString(); - } -} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicDescription.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicDescription.java deleted file mode 100644 index 443bb6828940..000000000000 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicDescription.java +++ /dev/null @@ -1,92 +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.kafka; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import static com.google.common.base.MoreObjects.toStringHelper; -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Strings.isNullOrEmpty; -import static java.util.Objects.requireNonNull; - -/** - * Json description to parse a row on a Kafka topic. A row contains a message and an optional key. See the documentation for the exact JSON syntax. - */ -public class KafkaTopicDescription -{ - private final String tableName; - private final String topicName; - private final String schemaName; - private final KafkaTopicFieldGroup key; - private final KafkaTopicFieldGroup message; - - @JsonCreator - public KafkaTopicDescription( - @JsonProperty("tableName") String tableName, - @JsonProperty("schemaName") String schemaName, - @JsonProperty("topicName") String topicName, - @JsonProperty("key") KafkaTopicFieldGroup key, - @JsonProperty("message") KafkaTopicFieldGroup message) - { - checkArgument(!isNullOrEmpty(tableName), "tableName is null or is empty"); - this.tableName = tableName; - this.topicName = requireNonNull(topicName, "topicName is null"); - this.schemaName = schemaName; - this.key = key; - this.message = message; - } - - @JsonProperty - public String getTableName() - { - return tableName; - } - - @JsonProperty - public String getTopicName() - { - return topicName; - } - - @JsonProperty - public String getSchemaName() - { - return schemaName; - } - - @JsonProperty - public KafkaTopicFieldGroup getKey() - { - return key; - } - - @JsonProperty - public KafkaTopicFieldGroup getMessage() - { - return message; - } - - @Override - public String toString() - { - return toStringHelper(this) - .add("tableName", tableName) - .add("topicName", topicName) - .add("schemaName", schemaName) - .add("key", key) - .add("message", message) - .toString(); - } -} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicFieldDescription.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicFieldDescription.java deleted file mode 100644 index 38c9861ea5f5..000000000000 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicFieldDescription.java +++ /dev/null @@ -1,159 +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.kafka; - -import com.facebook.presto.spi.ColumnMetadata; -import com.facebook.presto.spi.type.Type; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.util.Objects; - -import static com.google.common.base.MoreObjects.toStringHelper; -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Strings.isNullOrEmpty; -import static java.util.Objects.requireNonNull; - -/** - * Json description to parse a single field from a Kafka topic message. See {@link com.facebook.presto.kafka.KafkaTopicDescription} for more details. - */ -public final class KafkaTopicFieldDescription -{ - private final String name; - private final Type type; - private final String mapping; - private final String comment; - private final String dataFormat; - private final String formatHint; - private final boolean hidden; - - @JsonCreator - public KafkaTopicFieldDescription( - @JsonProperty("name") String name, - @JsonProperty("type") Type type, - @JsonProperty("mapping") String mapping, - @JsonProperty("comment") String comment, - @JsonProperty("dataFormat") String dataFormat, - @JsonProperty("formatHint") String formatHint, - @JsonProperty("hidden") boolean hidden) - { - checkArgument(!isNullOrEmpty(name), "name is null or is empty"); - this.name = name; - this.type = requireNonNull(type, "type is null"); - this.mapping = mapping; - this.comment = comment; - this.dataFormat = dataFormat; - this.formatHint = formatHint; - this.hidden = hidden; - } - - @JsonProperty - public String getName() - { - return name; - } - - @JsonProperty - public Type getType() - { - return type; - } - - @JsonProperty - public String getMapping() - { - return mapping; - } - - @JsonProperty - public String getComment() - { - return comment; - } - - @JsonProperty - public String getDataFormat() - { - return dataFormat; - } - - @JsonProperty - public String getFormatHint() - { - return formatHint; - } - - @JsonProperty - public boolean isHidden() - { - return hidden; - } - - KafkaColumnHandle getColumnHandle(String connectorId, boolean keyDecoder, int index) - { - return new KafkaColumnHandle(connectorId, - index, - getName(), - getType(), - getMapping(), - getDataFormat(), - getFormatHint(), - keyDecoder, - isHidden(), - false); - } - - ColumnMetadata getColumnMetadata() - { - return new ColumnMetadata(getName(), getType(), getComment(), isHidden()); - } - - @Override - public int hashCode() - { - return Objects.hash(name, type, mapping, dataFormat, formatHint, hidden); - } - - @Override - public boolean equals(Object obj) - { - if (this == obj) { - return true; - } - if (obj == null || getClass() != obj.getClass()) { - return false; - } - - KafkaTopicFieldDescription other = (KafkaTopicFieldDescription) obj; - return Objects.equals(this.name, other.name) && - Objects.equals(this.type, other.type) && - Objects.equals(this.mapping, other.mapping) && - Objects.equals(this.dataFormat, other.dataFormat) && - Objects.equals(this.formatHint, other.formatHint) && - Objects.equals(this.hidden, other.hidden); - } - - @Override - public String toString() - { - return toStringHelper(this) - .add("name", name) - .add("type", type) - .add("mapping", mapping) - .add("dataFormat", dataFormat) - .add("formatHint", formatHint) - .add("hidden", hidden) - .toString(); - } -} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicFieldGroup.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicFieldGroup.java deleted file mode 100644 index 4b5384fe982e..000000000000 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTopicFieldGroup.java +++ /dev/null @@ -1,62 +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.kafka; - -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.MoreObjects.toStringHelper; -import static java.util.Objects.requireNonNull; - -/** - * Groups the field descriptions for message or key. - */ -public class KafkaTopicFieldGroup -{ - private final String dataFormat; - private final List fields; - - @JsonCreator - public KafkaTopicFieldGroup( - @JsonProperty("dataFormat") String dataFormat, - @JsonProperty("fields") List fields) - { - this.dataFormat = requireNonNull(dataFormat, "dataFormat is null"); - this.fields = ImmutableList.copyOf(requireNonNull(fields, "fields is null")); - } - - @JsonProperty - public String getDataFormat() - { - return dataFormat; - } - - @JsonProperty - public List getFields() - { - return fields; - } - - @Override - public String toString() - { - return toStringHelper(this) - .add("dataFormat", dataFormat) - .add("fields", fields) - .toString(); - } -} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTransactionHandle.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTransactionHandle.java deleted file mode 100644 index f777be380ef2..000000000000 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaTransactionHandle.java +++ /dev/null @@ -1,22 +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.kafka; - -import com.facebook.presto.spi.connector.ConnectorTransactionHandle; - -public enum KafkaTransactionHandle - implements ConnectorTransactionHandle -{ - INSTANCE -} diff --git a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaUtil.java b/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaUtil.java deleted file mode 100644 index 4edf4f6da06f..000000000000 --- a/presto-kafka07/src/main/java/com/facebook/presto/kafka/KafkaUtil.java +++ /dev/null @@ -1,159 +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.kafka; - -import com.google.common.collect.Iterators; -import com.google.common.collect.Lists; -import kafka.cluster.Broker; -import kafka.cluster.Partition; -import kafka.consumer.ConsumerConfig; -import kafka.consumer.SimpleConsumer; -import kafka.utils.ZKConfig; -import kafka.utils.ZkUtils; -import org.I0Itec.zkclient.ZkClient; -import scala.collection.JavaConversions; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Properties; - -/** - * A collection of utility methods for accessing Kafka. - * - * @author Raghu Angadi - */ -public final class KafkaUtil -{ - public static final ConsumerConfig DEFAULT_CONSUMER_CONFIG; - - static { - Properties properties = new Properties(); - properties.setProperty("groupid", "this-should-not-be-used"); - DEFAULT_CONSUMER_CONFIG = new ConsumerConfig(properties); - } - - private KafkaUtil() - { - } - - /** - * create ZkClient with default options. - */ - public static ZkClient newZkClient(String zkConnect) - { - // get defaults from ZkConfig. - ZKConfig config = new ZKConfig(new Properties()); - - return new ZkClient(zkConnect, - config.zkSessionTimeoutMs(), - config.zkConnectionTimeoutMs(), - kafka.utils.ZKStringSerializer$.MODULE$); - } - - /** - * Returns partitions for given topic. An empty list if the topic is not - * found. - */ - public static List getPartitionsForTopic(ZkClient zkClient, - String topic) - { - // handle scala <-> java conversions. - scala.collection.Iterator topics = - JavaConversions.asScalaIterator(Iterators.forArray(topic)); - Map> map = - JavaConversions.mapAsJavaMap(ZkUtils.getPartitionsForTopics(zkClient, topics)); - - // since we are asking for just one topic, map's size is 0 or 1. - if (map.size() > 0) { - List partitions = JavaConversions.seqAsJavaList( - map.values().iterator().next()); - // transform string to Partition object - return Lists.newArrayList( - Lists.transform(partitions, - input -> Partition.parse(input))); - } - - return new ArrayList<>(); - } - - /** - * Returns latest offset before the given timestamp. If there is no offset - * avaliable, returns the earliest available. An offset before the timestamp - * may not be available if the messages are already rotated. - */ - public static long getBeforeOrEarliestOffset(SimpleConsumer consumer, - String topic, - int partId, - long time) - { - long[] offsets = consumer.getOffsetsBefore(topic, partId, time, 1); - if (offsets.length == 0) { - // then the earliest offset - offsets = consumer.getOffsetsBefore(topic, partId, -2, 1); - } - - return (offsets.length > 0) ? offsets[0] : 0; - } - - /** - * Returns the topics on given Kafka Server - */ - public static List getTopics(ZkClient zkClient) - { - String topicPath = ZkUtils.BrokerTopicsPath(); - return zkClient.getChildren(topicPath); - } - - /** - * Returns the brokers currently registered - */ - public static List getBrokersIds(ZkClient zkClient) - { - String brokerPath = ZkUtils.BrokerIdsPath(); - List brokers = zkClient.getChildren(brokerPath); - List brokerIds = new ArrayList(); - for (String s : brokers) { - Long l = Long.parseLong(s); - brokerIds.add(l); - } - - return brokerIds; - } - - /** - * Returns the number of partitions for a given topic and broker. - */ - public static Integer getNumPartitions(ZkClient zkClient, String topic, Long broker) - { - String topicPath = ZkUtils.BrokerTopicsPath(); - String partitionPath = topicPath + "/" + topic + "/" + broker.toString(); - - String numPartitions = zkClient.readData(partitionPath, true); - if (numPartitions == null) { - return 0; - } - else { - return Integer.parseInt(numPartitions); - } - } - - public static SimpleConsumer newSimpleConsumer(Broker broker) - { - return new SimpleConsumer(broker.host(), broker.port(), - DEFAULT_CONSUMER_CONFIG.socketTimeoutMs(), - DEFAULT_CONSUMER_CONFIG.socketBufferSize()); - } -} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/KafkaQueryRunner.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/KafkaQueryRunner.java deleted file mode 100644 index 1f760f8a4c46..000000000000 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/KafkaQueryRunner.java +++ /dev/null @@ -1,142 +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.kafka; - -import com.facebook.presto.Session; -import com.facebook.presto.kafka.util.CodecSupplier; -import com.facebook.presto.kafka.util.EmbeddedKafka; -import com.facebook.presto.kafka.util.TestUtils; -import com.facebook.presto.metadata.Metadata; -import com.facebook.presto.metadata.QualifiedObjectName; -import com.facebook.presto.spi.SchemaTableName; -import com.facebook.presto.tests.DistributedQueryRunner; -import com.facebook.presto.tests.TestingPrestoClient; -import com.facebook.presto.tpch.TpchPlugin; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import io.airlift.json.JsonCodec; -import io.airlift.log.Logger; -import io.airlift.log.Logging; -import io.airlift.tpch.TpchTable; - -import java.util.Map; - -import static com.facebook.presto.kafka.util.TestUtils.installKafkaPlugin; -import static com.facebook.presto.kafka.util.TestUtils.loadTpchTopicDescription; -import static com.facebook.presto.testing.TestingSession.testSessionBuilder; -import static com.facebook.presto.tpch.TpchMetadata.TINY_SCHEMA_NAME; -import static io.airlift.testing.Closeables.closeAllSuppress; -import static io.airlift.units.Duration.nanosSince; -import static java.util.Locale.ENGLISH; -import static java.util.concurrent.TimeUnit.SECONDS; - -public final class KafkaQueryRunner -{ - private KafkaQueryRunner() - { - } - - private static final Logger log = Logger.get("TestQueries"); - private static final String TPCH_SCHEMA = "tpch"; - - public static DistributedQueryRunner createKafkaQueryRunner(EmbeddedKafka embeddedKafka, TpchTable... tables) - throws Exception - { - return createKafkaQueryRunner(embeddedKafka, ImmutableList.copyOf(tables)); - } - - public static DistributedQueryRunner createKafkaQueryRunner(EmbeddedKafka embeddedKafka, Iterable> tables) - throws Exception - { - DistributedQueryRunner queryRunner = null; - try { - queryRunner = new DistributedQueryRunner(createSession(), 2); - - queryRunner.installPlugin(new TpchPlugin()); - queryRunner.createCatalog("tpch", "tpch"); - - embeddedKafka.start(); - - for (TpchTable table : tables) { - embeddedKafka.createTopics(kafkaTopicName(table)); - } - - Map topicDescriptions = createTpchTopicDescriptions(queryRunner.getCoordinator().getMetadata(), tables); - - installKafkaPlugin(embeddedKafka, queryRunner, topicDescriptions); - - TestingPrestoClient prestoClient = queryRunner.getClient(); - - log.info("Loading data..."); - long startTime = System.nanoTime(); - for (TpchTable table : tables) { - loadTpchTopic(embeddedKafka, prestoClient, table); - } - log.info("Loading complete in %s", nanosSince(startTime).toString(SECONDS)); - - return queryRunner; - } - catch (Throwable e) { - closeAllSuppress(e, queryRunner, embeddedKafka); - throw e; - } - } - - private static void loadTpchTopic(EmbeddedKafka embeddedKafka, TestingPrestoClient prestoClient, TpchTable table) - { - long start = System.nanoTime(); - log.info("Running import for %s", table.getTableName()); - TestUtils.loadTpchTopic(embeddedKafka, prestoClient, kafkaTopicName(table), new QualifiedObjectName("tpch", TINY_SCHEMA_NAME, table.getTableName().toLowerCase(ENGLISH))); - log.info("Imported %s in %s", 0, table.getTableName(), nanosSince(start).convertToMostSuccinctTimeUnit()); - } - - private static String kafkaTopicName(TpchTable table) - { - return TPCH_SCHEMA + "." + table.getTableName().toLowerCase(ENGLISH); - } - - private static Map createTpchTopicDescriptions(Metadata metadata, Iterable> tables) - throws Exception - { - JsonCodec topicDescriptionJsonCodec = new CodecSupplier<>(KafkaTopicDescription.class, metadata).get(); - - ImmutableMap.Builder topicDescriptions = ImmutableMap.builder(); - for (TpchTable table : tables) { - String tableName = table.getTableName(); - SchemaTableName tpchTable = new SchemaTableName(TPCH_SCHEMA, tableName); - - topicDescriptions.put(loadTpchTopicDescription(topicDescriptionJsonCodec, tpchTable.toString(), tpchTable)); - } - return topicDescriptions.build(); - } - - public static Session createSession() - { - return testSessionBuilder() - .setCatalog("kafka") - .setSchema(TPCH_SCHEMA) - .build(); - } - - public static void main(String[] args) - throws Exception - { - Logging.initialize(); - DistributedQueryRunner queryRunner = createKafkaQueryRunner(EmbeddedKafka.createEmbeddedKafka(), TpchTable.getTables()); - Thread.sleep(10); - Logger log = Logger.get(KafkaQueryRunner.class); - log.info("======== SERVER STARTED ========"); - log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl()); - } -} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaConnectorConfig.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaConnectorConfig.java deleted file mode 100644 index e824778a9bee..000000000000 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaConnectorConfig.java +++ /dev/null @@ -1,68 +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.kafka; - -import com.google.common.collect.ImmutableMap; -import io.airlift.configuration.testing.ConfigAssertions; -import org.testng.annotations.Test; - -import java.io.File; -import java.util.Map; - -public class TestKafkaConnectorConfig -{ - @Test - public void testDefaults() - { - ConfigAssertions.assertRecordedDefaults(ConfigAssertions.recordDefaults(KafkaConnectorConfig.class) - .setNodes("") - .setKafkaConnectTimeout("10s") - .setKafkaBufferSize("64kB") - .setDefaultSchema("default") - .setTableNames("") - .setTableDescriptionDir(new File("etc/kafka07/")) - .setHideInternalColumns(true) - .setFetchSize(10485760) - .setZkEndpoint("")); - } - - @Test - public void testExplicitPropertyMappings() - { - Map properties = new ImmutableMap.Builder() - .put("kafka.table-description-dir", "/var/lib/kafka") - .put("kafka.table-names", "table1, table2, table3") - .put("kafka.default-schema", "kafka") - .put("kafka.nodes", "localhost:12345,localhost:23456") - .put("kafka.connect-timeout", "1h") - .put("kafka.buffer-size", "1MB") - .put("kafka.hide-internal-columns", "false") - .put("kafka.fetch-size", "10000000") - .put("kafka.zk-endpoint", "localhost:2181") - .build(); - - KafkaConnectorConfig expected = new KafkaConnectorConfig() - .setTableDescriptionDir(new File("/var/lib/kafka")) - .setTableNames("table1, table2, table3") - .setDefaultSchema("kafka") - .setNodes("localhost:12345, localhost:23456") - .setKafkaConnectTimeout("1h") - .setKafkaBufferSize("1MB") - .setHideInternalColumns(false) - .setFetchSize(10000000) - .setZkEndpoint("localhost:2181"); - - ConfigAssertions.assertFullMapping(properties, expected); - } -} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaDistributed.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaDistributed.java deleted file mode 100644 index c31e40f5d62e..000000000000 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaDistributed.java +++ /dev/null @@ -1,52 +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.kafka; - -import com.facebook.presto.kafka.util.EmbeddedKafka; -import com.facebook.presto.tests.AbstractTestQueries; -import io.airlift.tpch.TpchTable; -import org.testng.annotations.AfterClass; -import org.testng.annotations.Test; - -import java.io.IOException; - -import static com.facebook.presto.kafka.KafkaQueryRunner.createKafkaQueryRunner; -import static com.facebook.presto.kafka.util.EmbeddedKafka.createEmbeddedKafka; - -@Test -public class TestKafkaDistributed - extends AbstractTestQueries -{ - private final EmbeddedKafka embeddedKafka; - - public TestKafkaDistributed() - throws Exception - { - this(createEmbeddedKafka()); - } - - public TestKafkaDistributed(EmbeddedKafka embeddedKafka) - throws Exception - { - super(() -> createKafkaQueryRunner(embeddedKafka, TpchTable.getTables())); - this.embeddedKafka = embeddedKafka; - } - - @AfterClass(alwaysRun = true) - public void destroy() - throws IOException - { - embeddedKafka.close(); - } -} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaIntegrationSmokeTest.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaIntegrationSmokeTest.java deleted file mode 100644 index faf02f0bfb80..000000000000 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaIntegrationSmokeTest.java +++ /dev/null @@ -1,52 +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.kafka; - -import com.facebook.presto.kafka.util.EmbeddedKafka; -import com.facebook.presto.tests.AbstractTestIntegrationSmokeTest; -import org.testng.annotations.AfterClass; -import org.testng.annotations.Test; - -import java.io.IOException; - -import static com.facebook.presto.kafka.KafkaQueryRunner.createKafkaQueryRunner; -import static com.facebook.presto.kafka.util.EmbeddedKafka.createEmbeddedKafka; -import static io.airlift.tpch.TpchTable.ORDERS; - -@Test -public class TestKafkaIntegrationSmokeTest - extends AbstractTestIntegrationSmokeTest -{ - private final EmbeddedKafka embeddedKafka; - - public TestKafkaIntegrationSmokeTest() - throws Exception - { - this(createEmbeddedKafka()); - } - - public TestKafkaIntegrationSmokeTest(EmbeddedKafka embeddedKafka) - throws Exception - { - super(() -> createKafkaQueryRunner(embeddedKafka, ORDERS)); - this.embeddedKafka = embeddedKafka; - } - - @AfterClass(alwaysRun = true) - public void destroy() - throws IOException - { - embeddedKafka.close(); - } -} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaPlugin.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaPlugin.java deleted file mode 100644 index ad5e19b0eef4..000000000000 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestKafkaPlugin.java +++ /dev/null @@ -1,46 +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.kafka; - -import com.facebook.presto.spi.connector.Connector; -import com.facebook.presto.spi.connector.ConnectorFactory; -import com.facebook.presto.testing.TestingConnectorContext; -import com.google.common.collect.ImmutableMap; -import org.testng.annotations.Test; - -import static com.google.common.collect.Iterables.getOnlyElement; -import static io.airlift.testing.Assertions.assertInstanceOf; -import static org.testng.Assert.assertNotNull; - -@Test -public class TestKafkaPlugin -{ - @Test - public void testSpinup() - { - KafkaPlugin plugin = new KafkaPlugin(); - - ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); - assertInstanceOf(factory, KafkaConnectorFactory.class); - - Connector c = factory.create( - "test-connector", - ImmutableMap.builder() - .put("kafka.table-names", "test") - .put("kafka.nodes", "localhost:9092") - .build(), - new TestingConnectorContext()); - assertNotNull(c); - } -} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestManySegments.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestManySegments.java deleted file mode 100644 index 56c39553fe76..000000000000 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestManySegments.java +++ /dev/null @@ -1,118 +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.kafka; - -import com.facebook.presto.Session; -import com.facebook.presto.kafka.util.EmbeddedKafka; -import com.facebook.presto.kafka.util.TestUtils; -import com.facebook.presto.spi.SchemaTableName; -import com.facebook.presto.spi.type.BigintType; -import com.facebook.presto.testing.MaterializedResult; -import com.facebook.presto.tests.StandaloneQueryRunner; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import kafka.javaapi.producer.ProducerData; -import org.testng.annotations.AfterClass; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; - -import java.util.Collections; -import java.util.Properties; -import java.util.UUID; - -import static com.facebook.presto.kafka.util.EmbeddedKafka.CloseableProducer; -import static com.facebook.presto.kafka.util.TestUtils.createEmptyTopicDescription; -import static com.facebook.presto.testing.TestingSession.testSessionBuilder; -import static com.facebook.presto.testing.assertions.Assert.assertEquals; - -@Test(singleThreaded = true) -public class TestManySegments -{ - private static final Session SESSION = testSessionBuilder() - .setCatalog("kafka") - .setSchema("default") - .build(); - - private EmbeddedKafka embeddedKafka; - private String topicName; - private StandaloneQueryRunner queryRunner; - - @BeforeClass - public void startKafka() - throws Exception - { - embeddedKafka = EmbeddedKafka.createEmbeddedKafka(); - embeddedKafka.start(); - - topicName = "test_" + UUID.randomUUID().toString().replaceAll("-", "_"); - - Properties topicProperties = new Properties(); - topicProperties.setProperty("segment.bytes", "1048576"); - - embeddedKafka.createTopics(1, 1, topicProperties, topicName); - - try (CloseableProducer producer = embeddedKafka.createProducer()) { - int jMax = 10_000; - int iMax = 100_000 / jMax; - for (long i = 0; i < iMax; i++) { - ImmutableList.Builder> builder = ImmutableList.builder(); - for (long j = 0; j < jMax; j++) { - builder.add(new ProducerData<>(topicName, i, Collections.singletonList(ImmutableMap.of("id", String.format("%05d", i * jMax + j), "value", UUID.randomUUID().toString())))); - } - producer.send(builder.build()); - } - } - } - - @AfterClass(alwaysRun = true) - public void stopKafka() - throws Exception - { - embeddedKafka.close(); - } - - @BeforeMethod - public void spinUp() - throws Exception - { - this.queryRunner = new StandaloneQueryRunner(SESSION); - - TestUtils.installKafkaPlugin(embeddedKafka, queryRunner, - ImmutableMap.builder() - .put(createEmptyTopicDescription(topicName, new SchemaTableName("default", topicName))) - .build()); - } - - @AfterMethod(alwaysRun = true) - public void tearDown() - throws Exception - { - queryRunner.close(); - } - - @Test - public void testManySegments() - throws Exception - { - MaterializedResult result = queryRunner.execute("SELECT count(_message) from " + topicName); - - MaterializedResult expected = MaterializedResult.resultBuilder(SESSION, BigintType.BIGINT) - .row(100000L) - .build(); - - assertEquals(result, expected); - } -} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestMinimalFunctionality.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestMinimalFunctionality.java deleted file mode 100644 index d26e77b4f9dc..000000000000 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/TestMinimalFunctionality.java +++ /dev/null @@ -1,147 +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.kafka; - -import com.facebook.presto.Session; -import com.facebook.presto.kafka.util.EmbeddedKafka; -import com.facebook.presto.kafka.util.TestUtils; -import com.facebook.presto.metadata.QualifiedObjectName; -import com.facebook.presto.metadata.TableHandle; -import com.facebook.presto.security.AllowAllAccessControl; -import com.facebook.presto.spi.SchemaTableName; -import com.facebook.presto.spi.type.BigintType; -import com.facebook.presto.testing.MaterializedResult; -import com.facebook.presto.tests.StandaloneQueryRunner; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import kafka.javaapi.producer.ProducerData; -import org.testng.annotations.AfterClass; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; - -import java.util.Collections; -import java.util.Optional; -import java.util.Properties; -import java.util.UUID; - -import static com.facebook.presto.kafka.util.EmbeddedKafka.CloseableProducer; -import static com.facebook.presto.kafka.util.TestUtils.createEmptyTopicDescription; -import static com.facebook.presto.testing.TestingSession.testSessionBuilder; -import static com.facebook.presto.testing.assertions.Assert.assertEquals; -import static com.facebook.presto.transaction.TransactionBuilder.transaction; -import static org.testng.Assert.assertTrue; - -@Test(singleThreaded = true) -public class TestMinimalFunctionality -{ - private static final Session SESSION = testSessionBuilder() - .setCatalog("kafka") - .setSchema("default") - .build(); - - private EmbeddedKafka embeddedKafka; - private String topicName; - private StandaloneQueryRunner queryRunner; - - @BeforeClass - public void startKafka() - throws Exception - { - embeddedKafka = EmbeddedKafka.createEmbeddedKafka(); - embeddedKafka.start(); - } - - @AfterClass - public void stopKafka() - throws Exception - { - embeddedKafka.close(); - } - - @BeforeMethod - public void spinUp() - throws Exception - { - this.topicName = "test_" + UUID.randomUUID().toString().replaceAll("-", "_"); - - Properties topicProperties = new Properties(); - embeddedKafka.createTopics(2, 1, topicProperties, topicName); - - this.queryRunner = new StandaloneQueryRunner(SESSION); - - TestUtils.installKafkaPlugin(embeddedKafka, queryRunner, - ImmutableMap.builder() - .put(createEmptyTopicDescription(topicName, new SchemaTableName("default", topicName))) - .build()); - } - - @AfterMethod - public void tearDown() - throws Exception - { - queryRunner.close(); - } - - private void createMessages(String topicName, int count) - { - try (CloseableProducer producer = embeddedKafka.createProducer()) { - ImmutableList.Builder> builder = ImmutableList.builder(); - for (long i = 0; i < count; i++) { - Object message = ImmutableMap.of("id", Long.toString(i), "value", UUID.randomUUID().toString()); - builder.add(new ProducerData<>(topicName, i, Collections.singletonList(message))); - } - producer.send(builder.build()); - } - } - - @Test - public void testTopicExists() - throws Exception - { - QualifiedObjectName name = new QualifiedObjectName("kafka", "default", topicName); - - transaction(queryRunner.getTransactionManager(), new AllowAllAccessControl()) - .singleStatement() - .execute(SESSION, session -> { - Optional handle = queryRunner.getServer().getMetadata().getTableHandle(session, name); - assertTrue(handle.isPresent()); - }); - } - - @Test - public void testTopicHasData() - throws Exception - { - MaterializedResult result = queryRunner.execute("SELECT count(1) from " + topicName); - - MaterializedResult expected = MaterializedResult.resultBuilder(SESSION, BigintType.BIGINT) - .row(0L) - .build(); - - assertEquals(result, expected); - - int count = 1000; - createMessages(topicName, count); - Thread.sleep(10_000); - result = queryRunner.execute("SELECT count(1) from " + topicName); - - expected = MaterializedResult.resultBuilder(SESSION, BigintType.BIGINT) - .row((long) count) - .build(); - - assertEquals(result, expected); - } -} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/CodecSupplier.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/CodecSupplier.java deleted file mode 100644 index 05f7a035869f..000000000000 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/CodecSupplier.java +++ /dev/null @@ -1,71 +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.kafka.util; - -import com.facebook.presto.metadata.Metadata; -import com.facebook.presto.spi.type.Type; -import com.fasterxml.jackson.databind.DeserializationContext; -import com.fasterxml.jackson.databind.deser.std.FromStringDeserializer; -import com.google.common.collect.ImmutableMap; -import io.airlift.json.JsonCodec; -import io.airlift.json.JsonCodecFactory; -import io.airlift.json.ObjectMapperProvider; - -import java.util.function.Supplier; - -import static com.facebook.presto.spi.type.TypeSignature.parseTypeSignature; - -public final class CodecSupplier - implements Supplier> -{ - private final Metadata metadata; - private final JsonCodecFactory codecFactory; - private final Class clazz; - - public CodecSupplier(Class clazz, Metadata metadata) - { - this.clazz = clazz; - this.metadata = metadata; - ObjectMapperProvider objectMapperProvider = new ObjectMapperProvider(); - objectMapperProvider.setJsonDeserializers(ImmutableMap.of(Type.class, new TypeDeserializer())); - this.codecFactory = new JsonCodecFactory(objectMapperProvider); - } - - @Override - public JsonCodec get() - { - return codecFactory.jsonCodec(clazz); - } - - private class TypeDeserializer - extends FromStringDeserializer - { - private static final long serialVersionUID = 1L; - - public TypeDeserializer() - { - super(Type.class); - } - - @Override - protected Type _deserialize(String value, DeserializationContext context) - { - Type type = metadata.getType(parseTypeSignature(value)); - if (type == null) { - throw new IllegalArgumentException(String.valueOf("Unknown type " + value)); - } - return type; - } - } -} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedKafka.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedKafka.java deleted file mode 100644 index 2ea3a57f3b48..000000000000 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedKafka.java +++ /dev/null @@ -1,166 +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.kafka.util; - -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Maps; -import com.google.common.io.Files; -import com.google.common.io.MoreFiles; -import kafka.javaapi.producer.Producer; -import kafka.producer.ProducerConfig; -import kafka.server.KafkaConfig; -import kafka.server.KafkaServerStartable; -import scala.Option; - -import java.io.Closeable; -import java.io.File; -import java.io.IOException; -import java.util.Map; -import java.util.Properties; -import java.util.concurrent.atomic.AtomicBoolean; - -import static com.facebook.presto.kafka.util.TestUtils.findUnusedPort; -import static com.facebook.presto.kafka.util.TestUtils.toProperties; -import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; -import static java.util.Objects.requireNonNull; - -public class EmbeddedKafka - implements Closeable -{ - private final EmbeddedZookeeper zookeeper; - private final int port; - private final File kafkaDataDir; - private final KafkaServerStartable kafka; - - private final AtomicBoolean started = new AtomicBoolean(); - private final AtomicBoolean stopped = new AtomicBoolean(); - - public static EmbeddedKafka createEmbeddedKafka() - throws IOException - { - return new EmbeddedKafka(new EmbeddedZookeeper(), new Properties()); - } - - public static EmbeddedKafka createEmbeddedKafka(Properties overrideProperties) - throws IOException - { - return new EmbeddedKafka(new EmbeddedZookeeper(), overrideProperties); - } - - EmbeddedKafka(EmbeddedZookeeper zookeeper, Properties overrideProperties) - throws IOException - { - this.zookeeper = requireNonNull(zookeeper, "zookeeper is null"); - requireNonNull(overrideProperties, "overrideProperties is null"); - - this.port = findUnusedPort(); - this.kafkaDataDir = Files.createTempDir(); - - Map properties = ImmutableMap.builder() - .put("brokerid", "0") - .put("regionid", "1") - .put("hostname", "localhost") - .put("num.partitions", "2") - .put("log.flush.interval", "10000") - .put("log.default.flush.interval.ms", "1000") - .put("log.retention.hours", "1") - .put("log.segment.bytes", "10248576") - .put("log.file.size", "1048576") - .put("auto.create.topics.enable", "false") - .put("zookeeper.connection.timeout.ms", "1000000") - .put("port", Integer.toString(port)) - .put("log.dir", kafkaDataDir.getAbsolutePath()) - .put("zookeeper.connect", zookeeper.getConnectString()) - .put("zk.connect", zookeeper.getConnectString()) - .putAll(Maps.fromProperties(overrideProperties)) - .build(); - - KafkaConfig config = new KafkaConfig(toProperties(properties)); - this.kafka = new KafkaServerStartable(config, Option.empty()); - } - - public void start() - throws InterruptedException, IOException - { - if (!started.getAndSet(true)) { - zookeeper.start(); - kafka.startup(); - } - } - - @Override - public void close() - throws IOException - { - if (started.get() && !stopped.getAndSet(true)) { - kafka.shutdown(); - kafka.awaitShutdown(); - zookeeper.close(); - MoreFiles.deleteRecursively(kafkaDataDir.toPath(), ALLOW_INSECURE); - } - } - - public void createTopics(String... topics) - { - createTopics(2, 1, new Properties(), topics); - } - - public void createTopics(int partitions, int replication, Properties topicProperties, String... topics) - { - // noop - } - - public CloseableProducer createProducer() - { - Map properties = ImmutableMap.builder() - .put("broker.list", String.format("0:%s", getConnectString())) - .put("serializer.class", JsonEncoder.class.getName()) - .put("key.serializer.class", NumberEncoder.class.getName()) - .put("request.required.acks", "1") - .build(); - - ProducerConfig producerConfig = new ProducerConfig(toProperties(properties)); - return new CloseableProducer<>(producerConfig); - } - - public static class CloseableProducer - extends Producer - implements AutoCloseable - { - public CloseableProducer(ProducerConfig config) - { - super(config); - } - } - - public int getZookeeperPort() - { - return zookeeper.getPort(); - } - - public int getPort() - { - return port; - } - - public String getConnectString() - { - return "localhost:" + Integer.toString(port); - } - - public String getZookeeperConnectString() - { - return zookeeper.getConnectString(); - } -} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedZookeeper.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedZookeeper.java deleted file mode 100644 index d312179421fd..000000000000 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/EmbeddedZookeeper.java +++ /dev/null @@ -1,101 +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.kafka.util; - -import com.google.common.io.Files; -import com.google.common.io.MoreFiles; -import org.apache.zookeeper.server.NIOServerCnxnFactory; -import org.apache.zookeeper.server.ServerCnxnFactory; -import org.apache.zookeeper.server.ZooKeeperServer; -import org.apache.zookeeper.server.persistence.FileTxnSnapLog; - -import java.io.Closeable; -import java.io.File; -import java.io.IOException; -import java.net.InetSocketAddress; -import java.util.concurrent.atomic.AtomicBoolean; - -import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; - -public class EmbeddedZookeeper - implements Closeable -{ - private final int port; - private final File zkDataDir; - private final ZooKeeperServer zkServer; - private final ServerCnxnFactory cnxnFactory; - - private final AtomicBoolean started = new AtomicBoolean(); - private final AtomicBoolean stopped = new AtomicBoolean(); - - public EmbeddedZookeeper() - throws IOException - { - this(TestUtils.findUnusedPort()); - } - - public EmbeddedZookeeper(int port) - throws IOException - { - this.port = port; - zkDataDir = Files.createTempDir(); - zkServer = new ZooKeeperServer(); - - FileTxnSnapLog ftxn = new FileTxnSnapLog(zkDataDir, zkDataDir); - zkServer.setTxnLogFactory(ftxn); - zkServer.setMinSessionTimeout(60000); - zkServer.setMaxSessionTimeout(120000); - - cnxnFactory = NIOServerCnxnFactory.createFactory(new InetSocketAddress(port), 300); - } - - public void start() - throws InterruptedException, IOException - { - if (!started.getAndSet(true)) { - cnxnFactory.startup(zkServer); - } - } - - @Override - public void close() - throws IOException - { - if (started.get() && !stopped.getAndSet(true)) { - cnxnFactory.shutdown(); - try { - cnxnFactory.join(); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - - if (zkServer.isRunning()) { - zkServer.shutdown(); - } - - MoreFiles.deleteRecursively(zkDataDir.toPath(), ALLOW_INSECURE); - } - } - - public String getConnectString() - { - return "127.0.0.1:" + Integer.toString(port); - } - - public int getPort() - { - return port; - } -} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/JsonEncoder.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/JsonEncoder.java deleted file mode 100644 index 65e54985c894..000000000000 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/JsonEncoder.java +++ /dev/null @@ -1,39 +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.kafka.util; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Throwables; -import kafka.message.Message; -import kafka.serializer.Encoder; - -import java.io.IOException; - -public class JsonEncoder - implements Encoder -{ - private final ObjectMapper objectMapper = new ObjectMapper(); - - @Override - public Message toMessage(Object o) - { - try { - return new Message(objectMapper.writeValueAsBytes(o)); - } - catch (IOException e) { - throw Throwables.propagate(e); - } - } -} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/KafkaLoader.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/KafkaLoader.java index f13fff45d305..e69de29bb2d1 100644 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/KafkaLoader.java +++ b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/KafkaLoader.java @@ -1,158 +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.kafka.util; - -import com.facebook.presto.Session; -import com.facebook.presto.client.Column; -import com.facebook.presto.client.QueryData; -import com.facebook.presto.client.QueryStatusInfo; -import com.facebook.presto.server.testing.TestingPrestoServer; -import com.facebook.presto.spi.type.TimeZoneKey; -import com.facebook.presto.spi.type.Type; -import com.facebook.presto.spi.type.Varchars; -import com.facebook.presto.tests.AbstractTestingPrestoClient; -import com.facebook.presto.tests.ResultsSession; -import com.google.common.collect.ImmutableMap; -import kafka.javaapi.producer.Producer; -import kafka.javaapi.producer.ProducerData; -import org.joda.time.format.DateTimeFormatter; -import org.joda.time.format.ISODateTimeFormat; - -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; - -import static com.facebook.presto.spi.type.BigintType.BIGINT; -import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; -import static com.facebook.presto.spi.type.DateTimeEncoding.unpackMillisUtc; -import static com.facebook.presto.spi.type.DateType.DATE; -import static com.facebook.presto.spi.type.DoubleType.DOUBLE; -import static com.facebook.presto.spi.type.IntegerType.INTEGER; -import static com.facebook.presto.spi.type.TimeType.TIME; -import static com.facebook.presto.spi.type.TimeWithTimeZoneType.TIME_WITH_TIME_ZONE; -import static com.facebook.presto.spi.type.TimestampType.TIMESTAMP; -import static com.facebook.presto.spi.type.TimestampWithTimeZoneType.TIMESTAMP_WITH_TIME_ZONE; -import static com.facebook.presto.util.DateTimeUtils.parseTimeLiteral; -import static com.facebook.presto.util.DateTimeUtils.parseTimestampWithTimeZone; -import static com.facebook.presto.util.DateTimeUtils.parseTimestampWithoutTimeZone; -import static com.google.common.base.Preconditions.checkState; -import static java.util.Objects.requireNonNull; - -public class KafkaLoader - extends AbstractTestingPrestoClient -{ - private static final DateTimeFormatter ISO8601_FORMATTER = ISODateTimeFormat.dateTime(); - - private final String topicName; - private final Producer producer; - private final AtomicLong count = new AtomicLong(); - - public KafkaLoader(Producer producer, - String topicName, - TestingPrestoServer prestoServer, - Session defaultSession) - { - super(prestoServer, defaultSession); - - this.topicName = topicName; - this.producer = producer; - } - - @Override - public ResultsSession getResultSession(Session session) - { - requireNonNull(session, "session is null"); - return new KafkaLoadingSession(session); - } - - private class KafkaLoadingSession - implements ResultsSession - { - private final AtomicReference> types = new AtomicReference<>(); - - private final TimeZoneKey timeZoneKey; - - private KafkaLoadingSession(Session session) - { - this.timeZoneKey = session.getTimeZoneKey(); - } - - @Override - public void addResults(QueryStatusInfo statusInfo, QueryData data) - { - if (types.get() == null && statusInfo.getColumns() != null) { - types.set(getTypes(statusInfo.getColumns())); - } - - if (data.getData() != null) { - checkState(types.get() != null, "Data without types received!"); - List columns = statusInfo.getColumns(); - for (List fields : data.getData()) { - ImmutableMap.Builder builder = ImmutableMap.builder(); - for (int i = 0; i < fields.size(); i++) { - Type type = types.get().get(i); - Object value = convertValue(fields.get(i), type); - if (value != null) { - builder.put(columns.get(i).getName(), value); - } - } - - producer.send(new ProducerData<>(topicName, count.getAndIncrement(), Collections.singletonList(builder.build()))); - } - } - } - - @Override - public Void build(Map setSessionProperties, Set resetSessionProperties) - { - return null; - } - - private Object convertValue(Object value, Type type) - { - if (value == null) { - return null; - } - - if (BOOLEAN.equals(type) || Varchars.isVarcharType(type)) { - return value; - } - if (BIGINT.equals(type)) { - return ((Number) value).longValue(); - } - if (INTEGER.equals(type)) { - return ((Number) value).intValue(); - } - if (DOUBLE.equals(type)) { - return ((Number) value).doubleValue(); - } - if (DATE.equals(type)) { - return value; - } - if (TIME.equals(type)) { - return ISO8601_FORMATTER.print(parseTimeLiteral(timeZoneKey, (String) value)); - } - if (TIMESTAMP.equals(type)) { - return ISO8601_FORMATTER.print(parseTimestampWithoutTimeZone(timeZoneKey, (String) value)); - } - if (TIME_WITH_TIME_ZONE.equals(type) || TIMESTAMP_WITH_TIME_ZONE.equals(type)) { - return ISO8601_FORMATTER.print(unpackMillisUtc(parseTimestampWithTimeZone(timeZoneKey, (String) value))); - } - throw new AssertionError("unhandled type: " + type); - } - } -} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberEncoder.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberEncoder.java deleted file mode 100644 index 07b627b4cff9..000000000000 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberEncoder.java +++ /dev/null @@ -1,31 +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.kafka.util; - -import kafka.message.Message; -import kafka.serializer.Encoder; - -import java.nio.ByteBuffer; - -public class NumberEncoder - implements Encoder -{ - @Override - public Message toMessage(Number value) - { - ByteBuffer buf = ByteBuffer.allocate(8); - buf.putLong(value == null ? 0L : value.longValue()); - return new Message(buf.array()); - } -} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberPartitioner.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberPartitioner.java deleted file mode 100644 index a1225b358813..000000000000 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/NumberPartitioner.java +++ /dev/null @@ -1,33 +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.kafka.util; - -import kafka.cluster.Partition; -import kafka.producer.Partitioner; -import scala.collection.Seq; - -import static java.lang.Math.toIntExact; - -public class NumberPartitioner - implements Partitioner -{ - @Override - public int partition(Object key, Seq partitions) - { - if (key instanceof Number) { - return toIntExact(((Number) key).longValue() % partitions.size()); - } - return 0; - } -} diff --git a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/TestUtils.java b/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/TestUtils.java deleted file mode 100644 index 5e194df60de6..000000000000 --- a/presto-kafka07/src/test/java/com/facebook/presto/kafka/util/TestUtils.java +++ /dev/null @@ -1,96 +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.kafka.util; - -import com.facebook.presto.kafka.KafkaPlugin; -import com.facebook.presto.kafka.KafkaTopicDescription; -import com.facebook.presto.metadata.QualifiedObjectName; -import com.facebook.presto.spi.SchemaTableName; -import com.facebook.presto.testing.QueryRunner; -import com.facebook.presto.tests.TestingPrestoClient; -import com.google.common.base.Joiner; -import com.google.common.collect.ImmutableMap; -import com.google.common.io.ByteStreams; -import io.airlift.json.JsonCodec; - -import java.io.IOException; -import java.net.ServerSocket; -import java.util.AbstractMap; -import java.util.Map; -import java.util.Properties; - -import static com.facebook.presto.kafka.util.EmbeddedKafka.CloseableProducer; -import static java.lang.String.format; - -public final class TestUtils -{ - private TestUtils() {} - - public static int findUnusedPort() - throws IOException - { - try (ServerSocket socket = new ServerSocket(0)) { - return socket.getLocalPort(); - } - } - - public static Properties toProperties(Map map) - { - Properties properties = new Properties(); - for (Map.Entry entry : map.entrySet()) { - properties.setProperty(entry.getKey(), entry.getValue()); - } - return properties; - } - - public static void installKafkaPlugin(EmbeddedKafka embeddedKafka, QueryRunner queryRunner, Map topicDescriptions) - { - KafkaPlugin kafkaPlugin = new KafkaPlugin(); - kafkaPlugin.setTableDescriptionSupplier(() -> topicDescriptions); - queryRunner.installPlugin(kafkaPlugin); - - Map kafkaConfig = ImmutableMap.of( - "kafka.nodes", embeddedKafka.getConnectString(), - "kafka.table-names", Joiner.on(",").join(topicDescriptions.keySet()), - "kafka.connect-timeout", "120s", - "kafka.default-schema", "default", - "kafka.zk-endpoint", embeddedKafka.getZookeeperConnectString()); - queryRunner.createCatalog("kafka", "kafka07", kafkaConfig); - } - - public static void loadTpchTopic(EmbeddedKafka embeddedKafka, TestingPrestoClient prestoClient, String topicName, QualifiedObjectName tpchTableName) - { - try (CloseableProducer producer = embeddedKafka.createProducer(); - KafkaLoader tpchLoader = new KafkaLoader(producer, topicName, prestoClient.getServer(), prestoClient.getDefaultSession())) { - tpchLoader.execute(format("SELECT * from %s", tpchTableName)); - } - } - - public static Map.Entry loadTpchTopicDescription(JsonCodec topicDescriptionJsonCodec, String topicName, SchemaTableName schemaTableName) - throws IOException - { - KafkaTopicDescription tpchTemplate = topicDescriptionJsonCodec.fromJson(ByteStreams.toByteArray(TestUtils.class.getResourceAsStream(format("/tpch/%s.json", schemaTableName.getTableName())))); - - return new AbstractMap.SimpleImmutableEntry<>( - schemaTableName, - new KafkaTopicDescription(schemaTableName.getTableName(), schemaTableName.getSchemaName(), topicName, tpchTemplate.getKey(), tpchTemplate.getMessage())); - } - - public static Map.Entry createEmptyTopicDescription(String topicName, SchemaTableName schemaTableName) - { - return new AbstractMap.SimpleImmutableEntry<>( - schemaTableName, - new KafkaTopicDescription(schemaTableName.getTableName(), schemaTableName.getSchemaName(), topicName, null, null)); - } -} diff --git a/presto-kafka07/src/test/resources/tpch/customer.json b/presto-kafka07/src/test/resources/tpch/customer.json deleted file mode 100644 index cf800fc0a9bb..000000000000 --- a/presto-kafka07/src/test/resources/tpch/customer.json +++ /dev/null @@ -1,61 +0,0 @@ -{ - "tableName": "customer", - "schemaName": "tpch", - "topicName": "tpch.customer", - "key": { - "dataFormat": "raw", - "fields": [ - { - "name": "kafka_key", - "dataFormat": "LONG", - "type": "BIGINT", - "hidden": "true" - } - ] - }, - "message": { - "dataFormat": "json", - "fields": [ - { - "name": "custkey", - "mapping": "custkey", - "type": "BIGINT" - }, - { - "name": "name", - "mapping": "name", - "type": "VARCHAR(25)" - }, - { - "name": "address", - "mapping": "address", - "type": "VARCHAR(40)" - }, - { - "name": "nationkey", - "mapping": "nationkey", - "type": "BIGINT" - }, - { - "name": "phone", - "mapping": "phone", - "type": "VARCHAR(15)" - }, - { - "name": "acctbal", - "mapping": "acctbal", - "type": "DOUBLE" - }, - { - "name": "mktsegment", - "mapping": "mktsegment", - "type": "VARCHAR(10)" - }, - { - "name": "comment", - "mapping": "comment", - "type": "VARCHAR(117)" - } - ] - } -} diff --git a/presto-kafka07/src/test/resources/tpch/lineitem.json b/presto-kafka07/src/test/resources/tpch/lineitem.json deleted file mode 100644 index 2250fddf7771..000000000000 --- a/presto-kafka07/src/test/resources/tpch/lineitem.json +++ /dev/null @@ -1,104 +0,0 @@ -{ - "tableName": "lineitem", - "schemaName": "tpch", - "topicName": "tpch.lineitem", - "key": { - "dataFormat": "raw", - "fields": [ - { - "name": "kafka_key", - "dataFormat": "LONG", - "type": "BIGINT", - "hidden": "true" - } - ] - }, - "message": { - "dataFormat": "json", - "fields": [ - { - "name": "orderkey", - "mapping": "orderkey", - "type": "BIGINT" - }, - { - "name": "partkey", - "mapping": "partkey", - "type": "BIGINT" - }, - { - "name": "suppkey", - "mapping": "suppkey", - "type": "BIGINT" - }, - { - "name": "linenumber", - "mapping": "linenumber", - "type": "INTEGER" - }, - { - "name": "quantity", - "mapping": "quantity", - "type": "DOUBLE" - }, - { - "name": "extendedprice", - "mapping": "extendedprice", - "type": "DOUBLE" - }, - { - "name": "discount", - "mapping": "discount", - "type": "DOUBLE" - }, - { - "name": "tax", - "mapping": "tax", - "type": "DOUBLE" - }, - { - "name": "returnflag", - "mapping": "returnflag", - "type": "VARCHAR(1)" - }, - { - "name": "linestatus", - "mapping": "linestatus", - "type": "VARCHAR(1)" - }, - { - "name": "shipdate", - "mapping": "shipdate", - "type": "DATE", - "dataFormat": "iso8601" - }, - { - "name": "commitdate", - "mapping": "commitdate", - "type": "DATE", - "dataFormat": "iso8601" - }, - { - "name": "receiptdate", - "mapping": "receiptdate", - "type": "DATE", - "dataFormat": "iso8601" - }, - { - "name": "shipinstruct", - "mapping": "shipinstruct", - "type": "VARCHAR(25)" - }, - { - "name": "shipmode", - "mapping": "shipmode", - "type": "VARCHAR(7)" - }, - { - "name": "comment", - "mapping": "comment", - "type": "VARCHAR(44)" - } - ] - } -} diff --git a/presto-kafka07/src/test/resources/tpch/nation.json b/presto-kafka07/src/test/resources/tpch/nation.json deleted file mode 100644 index 0749a1249d76..000000000000 --- a/presto-kafka07/src/test/resources/tpch/nation.json +++ /dev/null @@ -1,41 +0,0 @@ -{ - "tableName": "nation", - "schemaName": "tpch", - "topicName": "tpch.nation", - "key": { - "dataFormat": "raw", - "fields": [ - { - "name": "kafka_key", - "dataFormat": "LONG", - "type": "BIGINT", - "hidden": "true" - } - ] - }, - "message": { - "dataFormat": "json", - "fields": [ - { - "name": "nationkey", - "mapping": "nationkey", - "type": "BIGINT" - }, - { - "name": "name", - "mapping": "name", - "type": "VARCHAR(25)" - }, - { - "name": "regionkey", - "mapping": "regionkey", - "type": "BIGINT" - }, - { - "name": "comment", - "mapping": "comment", - "type": "VARCHAR(152)" - } - ] - } -} diff --git a/presto-kafka07/src/test/resources/tpch/orders.json b/presto-kafka07/src/test/resources/tpch/orders.json deleted file mode 100644 index f5eb594083a4..000000000000 --- a/presto-kafka07/src/test/resources/tpch/orders.json +++ /dev/null @@ -1,67 +0,0 @@ -{ - "tableName": "orders", - "schemaName": "tpch", - "topicName": "tpch.orders", - "key": { - "dataFormat": "raw", - "fields": [ - { - "name": "kafka_key", - "dataFormat": "LONG", - "type": "BIGINT", - "hidden": "true" - } - ] - }, - "message": { - "dataFormat": "json", - "fields": [ - { - "name": "orderkey", - "mapping": "orderkey", - "type": "BIGINT" - }, - { - "name": "custkey", - "mapping": "custkey", - "type": "BIGINT" - }, - { - "name": "orderstatus", - "mapping": "orderstatus", - "type": "VARCHAR(1)" - }, - { - "name": "totalprice", - "mapping": "totalprice", - "type": "DOUBLE" - }, - { - "name": "orderdate", - "mapping": "orderdate", - "type": "DATE", - "dataFormat": "iso8601" - }, - { - "name": "orderpriority", - "mapping": "orderpriority", - "type": "VARCHAR(15)" - }, - { - "name": "clerk", - "mapping": "clerk", - "type": "VARCHAR(15)" - }, - { - "name": "shippriority", - "mapping": "shippriority", - "type": "INTEGER" - }, - { - "name": "comment", - "mapping": "comment", - "type": "VARCHAR(79)" - } - ] - } -} diff --git a/presto-kafka07/src/test/resources/tpch/part.json b/presto-kafka07/src/test/resources/tpch/part.json deleted file mode 100644 index 3c56b27955f1..000000000000 --- a/presto-kafka07/src/test/resources/tpch/part.json +++ /dev/null @@ -1,66 +0,0 @@ -{ - "tableName": "part", - "schemaName": "tpch", - "topicName": "tpch.part", - "key": { - "dataFormat": "raw", - "fields": [ - { - "name": "kafka_key", - "dataFormat": "LONG", - "type": "BIGINT", - "hidden": "true" - } - ] - }, - "message": { - "dataFormat": "json", - "fields": [ - { - "name": "partkey", - "mapping": "partkey", - "type": "BIGINT" - }, - { - "name": "name", - "mapping": "name", - "type": "VARCHAR(55)" - }, - { - "name": "mfgr", - "mapping": "mfgr", - "type": "VARCHAR(25)" - }, - { - "name": "brand", - "mapping": "brand", - "type": "VARCHAR(10)" - }, - { - "name": "type", - "mapping": "type", - "type": "VARCHAR(25)" - }, - { - "name": "size", - "mapping": "size", - "type": "INTEGER" - }, - { - "name": "container", - "mapping": "container", - "type": "VARCHAR(10)" - }, - { - "name": "retailprice", - "mapping": "retailprice", - "type": "DOUBLE" - }, - { - "name": "comment", - "mapping": "comment", - "type": "VARCHAR(23)" - } - ] - } -} diff --git a/presto-kafka07/src/test/resources/tpch/partsupp.json b/presto-kafka07/src/test/resources/tpch/partsupp.json deleted file mode 100644 index cd6d6364f87f..000000000000 --- a/presto-kafka07/src/test/resources/tpch/partsupp.json +++ /dev/null @@ -1,46 +0,0 @@ -{ - "tableName": "partsupp", - "schemaName": "tpch", - "topicName": "tpch.partsupp", - "key": { - "dataFormat": "raw", - "fields": [ - { - "name": "kafka_key", - "dataFormat": "LONG", - "type": "BIGINT", - "hidden": "true" - } - ] - }, - "message": { - "dataFormat": "json", - "fields": [ - { - "name": "partkey", - "mapping": "partkey", - "type": "BIGINT" - }, - { - "name": "suppkey", - "mapping": "suppkey", - "type": "BIGINT" - }, - { - "name": "availqty", - "mapping": "availqty", - "type": "INTEGER" - }, - { - "name": "supplycost", - "mapping": "supplycost", - "type": "DOUBLE" - }, - { - "name": "comment", - "mapping": "comment", - "type": "VARCHAR(199)" - } - ] - } -} diff --git a/presto-kafka07/src/test/resources/tpch/region.json b/presto-kafka07/src/test/resources/tpch/region.json deleted file mode 100644 index 4aca016e22cc..000000000000 --- a/presto-kafka07/src/test/resources/tpch/region.json +++ /dev/null @@ -1,36 +0,0 @@ -{ - "tableName": "region", - "schemaName": "tpch", - "topicName": "tpch.region", - "key": { - "dataFormat": "raw", - "fields": [ - { - "name": "kafka_key", - "dataFormat": "LONG", - "type": "BIGINT", - "hidden": "true" - } - ] - }, - "message": { - "dataFormat": "json", - "fields": [ - { - "name": "regionkey", - "mapping": "regionkey", - "type": "BIGINT" - }, - { - "name": "name", - "mapping": "name", - "type": "VARCHAR(25)" - }, - { - "name": "comment", - "mapping": "comment", - "type": "VARCHAR(152)" - } - ] - } -} diff --git a/presto-kafka07/src/test/resources/tpch/supplier.json b/presto-kafka07/src/test/resources/tpch/supplier.json deleted file mode 100644 index af9d24c86a83..000000000000 --- a/presto-kafka07/src/test/resources/tpch/supplier.json +++ /dev/null @@ -1,56 +0,0 @@ -{ - "tableName": "supplier", - "schemaName": "tpch", - "topicName": "tpch.supplier", - "key": { - "dataFormat": "raw", - "fields": [ - { - "name": "kafka_key", - "dataFormat": "LONG", - "type": "BIGINT", - "hidden": "true" - } - ] - }, - "message": { - "dataFormat": "json", - "fields": [ - { - "name": "suppkey", - "mapping": "suppkey", - "type": "BIGINT" - }, - { - "name": "name", - "mapping": "name", - "type": "VARCHAR(25)" - }, - { - "name": "address", - "mapping": "address", - "type": "VARCHAR(40)" - }, - { - "name": "nationkey", - "mapping": "nationkey", - "type": "BIGINT" - }, - { - "name": "phone", - "mapping": "phone", - "type": "VARCHAR(15)" - }, - { - "name": "acctbal", - "mapping": "acctbal", - "type": "DOUBLE" - }, - { - "name": "comment", - "mapping": "comment", - "type": "VARCHAR(101)" - } - ] - } -} diff --git a/presto-main/etc/config.properties b/presto-main/etc/config.properties index d275748712fa..a84531ecb4e8 100644 --- a/presto-main/etc/config.properties +++ b/presto-main/etc/config.properties @@ -34,7 +34,6 @@ plugin.bundles=\ ../presto-hive-hadoop2/pom.xml,\ ../presto-example-http/pom.xml,\ ../presto-kafka/pom.xml, \ - ../presto-kafka07/pom.xml, \ ../presto-tpch/pom.xml, \ ../presto-local-file/pom.xml, \ ../presto-mysql/pom.xml,\ diff --git a/presto-twitter-server/src/main/provisio/twitter.xml b/presto-twitter-server/src/main/provisio/twitter.xml index e1da085284c0..d3660fa5e09b 100644 --- a/presto-twitter-server/src/main/provisio/twitter.xml +++ b/presto-twitter-server/src/main/provisio/twitter.xml @@ -2,12 +2,6 @@ - - - - - - From 21b6a89c8b5b0e08d7a8f594e56e47ddf2e0ed2b Mon Sep 17 00:00:00 2001 From: Mainak Ghosh Date: Thu, 13 Sep 2018 17:41:50 -0700 Subject: [PATCH 299/331] Updating travis file to redistribute test load --- .travis.yml | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/.travis.yml b/.travis.yml index 1054d8d85a50..2bee54ddbb49 100644 --- a/.travis.yml +++ b/.travis.yml @@ -21,6 +21,7 @@ env: - PRODUCT_TESTS_BASIC_ENVIRONMENT=true - PRODUCT_TESTS_SPECIFIC_ENVIRONMENT=true - PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_2=true + - PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_3=true - HIVE_TESTS=true - KUDU_TESTS=true @@ -61,7 +62,7 @@ install: ./mvnw install $MAVEN_FAST_INSTALL -P !twitter-modules -pl '!presto-docs,!presto-server,!presto-server-rpm' fi - | - if [[ -v PRODUCT_TESTS_BASIC_ENVIRONMENT || -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT || -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_2 ]]; then + if [[ -v PRODUCT_TESTS_BASIC_ENVIRONMENT || -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT || -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_2 || -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_3 ]]; then ./mvnw install $MAVEN_FAST_INSTALL -P !twitter-modules -pl '!presto-docs,!presto-server-rpm' fi - | @@ -128,7 +129,7 @@ script: singlenode-kerberos-hdfs-impersonation -g storage_formats,cli,hdfs_impersonation,authorization,hive_file_header fi - | - if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then + if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_3 ]]; then presto-product-tests/bin/run_on_docker.sh \ singlenode-kerberos-hdfs-impersonation-cross-realm -g storage_formats,cli,hdfs_impersonation fi From 6d439511843ae1342ee22014dc303812b429e1ae Mon Sep 17 00:00:00 2001 From: Hao Luo Date: Mon, 17 Sep 2018 11:48:44 -0700 Subject: [PATCH 300/331] use latest aircompressor in parquet compression utils --- pom.xml | 3 +- .../hive/parquet/HadoopLzoDecompressor.java | 69 ------------------- .../hive/parquet/ParquetCompressionUtils.java | 3 +- 3 files changed, 4 insertions(+), 71 deletions(-) delete mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/parquet/HadoopLzoDecompressor.java diff --git a/pom.xml b/pom.xml index ffbaac06cfaf..bf1ce3990949 100644 --- a/pom.xml +++ b/pom.xml @@ -414,10 +414,11 @@ 0.8.2 + io.airlift aircompressor - 0.11 + 0.12-SNAPSHOT diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/HadoopLzoDecompressor.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/HadoopLzoDecompressor.java deleted file mode 100644 index a87b648ca14b..000000000000 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/HadoopLzoDecompressor.java +++ /dev/null @@ -1,69 +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.parquet; - -import com.hadoop.compression.lzo.LzoCodec; -import io.airlift.compress.Decompressor; -import io.airlift.compress.MalformedInputException; -import org.apache.hadoop.conf.Configuration; - -import java.io.IOException; -import java.nio.ByteBuffer; - -import static com.google.common.base.Preconditions.checkArgument; - -// HadoopLzoDecompressor from aircompressor -public class HadoopLzoDecompressor - implements Decompressor -{ - private static final Configuration HADOOP_CONF = new Configuration(); - - private final org.apache.hadoop.io.compress.Decompressor decompressor; - - public HadoopLzoDecompressor() - { - LzoCodec codec = new LzoCodec(); - codec.setConf(HADOOP_CONF); - decompressor = codec.createDecompressor(); - } - - @Override - public int decompress(byte[] input, int inputOffset, int inputLength, byte[] output, int outputOffset, int maxOutputLength) - throws MalformedInputException - { - decompressor.reset(); - decompressor.setInput(input, inputOffset, inputLength); - - int offset = outputOffset; - int outputLimit = outputOffset + maxOutputLength; - while (!decompressor.finished() && offset < outputLimit) { - try { - offset += decompressor.decompress(output, offset, outputLimit - offset); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - checkArgument(decompressor.getRemaining() == 0); - return offset - outputOffset; - } - - @Override - public void decompress(ByteBuffer input, ByteBuffer output) - throws MalformedInputException - { - throw new UnsupportedOperationException("Not supported in HadoopLzoDecompressor"); - } -} diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetCompressionUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetCompressionUtils.java index 963f432e408b..ef9a300a4acd 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetCompressionUtils.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetCompressionUtils.java @@ -14,6 +14,7 @@ package com.facebook.presto.hive.parquet; import io.airlift.compress.Decompressor; +import io.airlift.compress.lzo.LzoDecompressor; import io.airlift.compress.snappy.SnappyDecompressor; import io.airlift.slice.DynamicSliceOutput; import io.airlift.slice.Slice; @@ -87,7 +88,7 @@ private static Slice decompressGzip(Slice input, int uncompressedSize) private static Slice decompressLZO(Slice input, int uncompressedSize) { - Decompressor lzoDecompressor = new HadoopLzoDecompressor(); + LzoDecompressor lzoDecompressor = new LzoDecompressor(); long totalDecompressedCount = 0; // over allocate buffer which makes decompression easier byte[] output = new byte[uncompressedSize + SIZE_OF_LONG]; From 7ab39fe2df09a3a3284255e950c2799f770eb3b7 Mon Sep 17 00:00:00 2001 From: Hao Luo Date: Mon, 17 Sep 2018 18:45:17 -0700 Subject: [PATCH 301/331] update travis yml --- .travis.yml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/.travis.yml b/.travis.yml index b68bd71770e6..ba61b36a8024 100644 --- a/.travis.yml +++ b/.travis.yml @@ -48,6 +48,10 @@ before_install: if [[ ! -e /usr/local/bin/thrift ]]; then sudo ln -s $HOME/.thrift/bin/thrift /usr/local/bin/thrift fi + - | + git clone https://github.com/airlift/aircompressor.git + cd aircompressor && mvn clean install -DskipTests + cd .. install: - ./mvnw -v From a23bfad6d82d68bd9931a206d6c7eb74cf034e65 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 19 Sep 2018 11:46:35 -0700 Subject: [PATCH 302/331] Add the option to by pass the authentication filter (#172) --- .../server/security/AuthenticationFilter.java | 8 ++++++++ .../presto/server/security/SecurityConfig.java | 15 +++++++++++++++ .../server/security/TestSecurityConfig.java | 7 +++++-- 3 files changed, 28 insertions(+), 2 deletions(-) diff --git a/presto-main/src/main/java/com/facebook/presto/server/security/AuthenticationFilter.java b/presto-main/src/main/java/com/facebook/presto/server/security/AuthenticationFilter.java index fcb691b46ace..9381da12d67c 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/security/AuthenticationFilter.java +++ b/presto-main/src/main/java/com/facebook/presto/server/security/AuthenticationFilter.java @@ -45,12 +45,14 @@ public class AuthenticationFilter { private final List authenticators; private final String httpAuthenticationPathRegex; + private final boolean allowByPass; @Inject public AuthenticationFilter(Set authenticators, SecurityConfig securityConfig) { this.authenticators = ImmutableList.copyOf(authenticators); this.httpAuthenticationPathRegex = requireNonNull(securityConfig.getHttpAuthenticationPathRegex(), "httpAuthenticationPathRegex is null"); + this.allowByPass = securityConfig.getAllowByPass(); } @Override @@ -94,6 +96,12 @@ public void doFilter(ServletRequest servletRequest, ServletResponse servletRespo return; } + // if authentication by pass allowed. + if (allowByPass) { + nextFilter.doFilter(request, response); + return; + } + // authentication failed skipRequestBody(request); diff --git a/presto-main/src/main/java/com/facebook/presto/server/security/SecurityConfig.java b/presto-main/src/main/java/com/facebook/presto/server/security/SecurityConfig.java index ec92f0dbccf7..bf275b64a110 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/security/SecurityConfig.java +++ b/presto-main/src/main/java/com/facebook/presto/server/security/SecurityConfig.java @@ -35,6 +35,8 @@ public class SecurityConfig private String httpAuthenticationPathRegex = "^\b$"; + private boolean allowByPass; + public enum AuthenticationType { CERTIFICATE, @@ -83,4 +85,17 @@ public SecurityConfig setHttpAuthenticationPathRegex(String regex) httpAuthenticationPathRegex = regex; return this; } + + public boolean getAllowByPass() + { + return allowByPass; + } + + @Config("http-server.authentication.allow-by-pass") + @ConfigDescription("Allow authentication by pass") + public SecurityConfig setAllowByPass(boolean allowByPass) + { + this.allowByPass = allowByPass; + return this; + } } diff --git a/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java b/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java index 52064739490e..777360358857 100644 --- a/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java +++ b/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java @@ -30,7 +30,8 @@ public void testDefaults() { ConfigAssertions.assertRecordedDefaults(ConfigAssertions.recordDefaults(SecurityConfig.class) .setAuthenticationTypes("") - .setHttpAuthenticationPathRegex("^\b$")); + .setHttpAuthenticationPathRegex("^\b$") + .setAllowByPass(false)); } @Test @@ -39,11 +40,13 @@ public void testExplicitPropertyMappings() Map properties = new ImmutableMap.Builder() .put("http-server.authentication.type", "KERBEROS,PASSWORD") .put("http-server.http.authentication.path.regex", "^/v1/statement") + .put("http-server.authentication.allow-by-pass", "true") .build(); SecurityConfig expected = new SecurityConfig() .setAuthenticationTypes(ImmutableList.of(KERBEROS, PASSWORD)) - .setHttpAuthenticationPathRegex("^/v1/statement"); + .setHttpAuthenticationPathRegex("^/v1/statement") + .setAllowByPass(true); ConfigAssertions.assertFullMapping(properties, expected); } From 089ac99bad2ac36f720da4a3aaa950cc6c4138ff Mon Sep 17 00:00:00 2001 From: Mainak Ghosh Date: Wed, 19 Sep 2018 22:19:34 -0700 Subject: [PATCH 303/331] Fixing the crash while emitting metrics. JIRA IQ 1274 --- .../twitter/presto/plugin/eventlistener/QueryStatsHelper.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java index b38c5c90e9f3..7f5456dce2ae 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java @@ -83,7 +83,6 @@ private static QueryStageInfo getQueryStageInfo(int stageId, JsonObject stage) stageInfo.peak_memory_reservation_bytes = getBytesOrNegativeOne(stageStats.getString("peakUserMemoryReservation")); stageInfo.total_scheduled_time_millis = getMillisOrNegativeOne(stageStats.getString("totalScheduledTime")); stageInfo.total_cpu_time_millis = getMillisOrNegativeOne(stageStats.getString("totalCpuTime")); - stageInfo.total_user_time_millis = getMillisOrNegativeOne(stageStats.getString("totalUserTime")); stageInfo.total_blocked_time_millis = getMillisOrNegativeOne(stageStats.getString("totalBlockedTime")); } catch (Exception e) { @@ -120,21 +119,18 @@ private static OperatorStats getOperatorStat(JsonObject obj) operatorStats.add_input_calls = obj.getJsonNumber("addInputCalls").longValue(); operatorStats.add_input_wall_millis = getMillisOrNegativeOne(obj.getString("addInputWall")); operatorStats.add_input_cpu_millis = getMillisOrNegativeOne(obj.getString("addInputCpu")); - operatorStats.add_input_user_millis = getMillisOrNegativeOne(obj.getString("addInputUser")); operatorStats.input_data_size_bytes = getBytesOrNegativeOne(obj.getString("inputDataSize")); operatorStats.input_positions = obj.getJsonNumber("inputPositions").longValue(); operatorStats.sum_squared_input_positions = obj.getJsonNumber("sumSquaredInputPositions").doubleValue(); operatorStats.get_output_calls = obj.getJsonNumber("getOutputCalls").longValue(); operatorStats.get_output_wall_millis = getMillisOrNegativeOne(obj.getString("getOutputWall")); operatorStats.get_output_cpu_millis = getMillisOrNegativeOne(obj.getString("getOutputCpu")); - operatorStats.get_output_user_millis = getMillisOrNegativeOne(obj.getString("getOutputUser")); operatorStats.output_data_size_bytes = getBytesOrNegativeOne(obj.getString("outputDataSize")); operatorStats.output_positions = obj.getJsonNumber("outputPositions").longValue(); operatorStats.blocked_wall_millis = getMillisOrNegativeOne(obj.getString("blockedWall")); operatorStats.finish_calls = obj.getJsonNumber("finishCalls").longValue(); operatorStats.finish_wall_millis = getMillisOrNegativeOne(obj.getString("finishWall")); operatorStats.finish_cpu_millis = getMillisOrNegativeOne(obj.getString("finishCpu")); - operatorStats.finish_user_millis = getMillisOrNegativeOne(obj.getString("finishUser")); operatorStats.memory_reservation_bytes = getBytesOrNegativeOne(obj.getString("userMemoryReservation")); operatorStats.system_memory_reservation_bytes = getBytesOrNegativeOne(obj.getString("systemMemoryReservation")); } From 22b3a513ed7e91e067d7405d23a01f29f0b72a3c Mon Sep 17 00:00:00 2001 From: Mainak Ghosh Date: Wed, 19 Sep 2018 22:21:44 -0700 Subject: [PATCH 304/331] Updating minor revision. 0.210-tw-0.55 --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial-toolkit/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kudu/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory-context/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-password-authenticators/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-proxy/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 4 ++-- 59 files changed, 61 insertions(+), 61 deletions(-) diff --git a/pom.xml b/pom.xml index 6732c23da111..91a6700b18d3 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.210-tw-0.54 + 0.210-tw-0.55 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 6124dfc98e4a..34453256f28e 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 03ae4ef1983b..dbc1c61d1242 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index e3546de12f6a..2b573d676c5a 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 9637425d9e7b..6df2d874526c 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index b0b6cd3fbb09..0548b7a47de8 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 13352b3bf90c..78ad5af35d58 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.54 + 0.210-tw-0.55 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 503e6718cd9a..692a76156f42 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index ceafe86852d6..75f9f6efe990 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index f14fafe926ca..bc0b1aeb6c9a 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 4d2a1ed4f6a3..44664fd28700 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 3798fdf8c2a4..64c22f4a0864 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 8dbd7a85e6d4..3a6ac2c520a5 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index c8144b9fc17e..0d91f6862b14 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index 03b2e6e99f4a..47a9b095269b 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index 89fee3645d72..1b67c90c483a 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 478b5fda4c3a..53ecd60b6090 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index c5579464a3c0..eb863f53027a 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 4b2f8170e3f4..eae0ec10ba98 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 3d18300427a1..0459524f1391 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 9430a6eab4a5..a0aa4bf326c0 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-kafka diff --git a/presto-kudu/pom.xml b/presto-kudu/pom.xml index f009804af96b..15702c14833b 100644 --- a/presto-kudu/pom.xml +++ b/presto-kudu/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-kudu diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 8f7633e438ff..ae95b1d2a403 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 956c2df3b8a7..1a0e44caf41e 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index 092fdfd6029f..2f9223efd378 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.210-tw-0.54 + 0.210-tw-0.55 presto-matching diff --git a/presto-memory-context/pom.xml b/presto-memory-context/pom.xml index 3720372ceb70..b85c292c73a0 100644 --- a/presto-memory-context/pom.xml +++ b/presto-memory-context/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-memory-context diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index cd26bd8134ee..14ce9dc2bfd9 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index a6dc8b6efeb2..11944938921d 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 33dba144c3c0..81ab3246b520 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 39bb86b617f0..1d91d0255e42 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 190679660fb4..7eff22426adc 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index af24570e9127..396cfbec7dcc 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-parser diff --git a/presto-password-authenticators/pom.xml b/presto-password-authenticators/pom.xml index a9448b1f7693..3777f48ce22d 100644 --- a/presto-password-authenticators/pom.xml +++ b/presto-password-authenticators/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-password-authenticators diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index 3e0212a33f1b..ea7e4bcbae3e 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index e115f2090da5..a8f1a239eaaa 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index aa0da27b1dd8..d40aa9dac9e0 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.54 + 0.210-tw-0.55 presto-product-tests diff --git a/presto-proxy/pom.xml b/presto-proxy/pom.xml index a2d3ef10e5e4..941460638859 100644 --- a/presto-proxy/pom.xml +++ b/presto-proxy/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-proxy diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 92e927fe699e..6508c662cf64 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index b235158f1134..5123951ab9ec 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 0a9f34fdcca6..74568ddb6139 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index bf4b706e0c52..f2a4c3977341 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index 4129a9394afb..bfa8171aab26 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 917d7588b2e5..d6a2efd03946 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index dbe0762926af..a48a4e0af2aa 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index b159790e160e..d6c5d420c31b 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index f55ac7bdfb2b..83e2f2308ee3 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 098ff2301029..0545e6373a7d 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.210-tw-0.54 + 0.210-tw-0.55 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 632abdee3ca6..491ce4dc81d1 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 81bc8a59f7fc..0663ea5855f9 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 1e2210835a27..552656cc1570 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.54 + 0.210-tw-0.55 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index e7b43e32fbb4..05de0e7f5cfb 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 96c310ae4570..e652d9dbac85 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index d3456b9f1859..97b06cb62fb4 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index 62846ea430ad..ffa8da1e0e1f 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 6268ac964ffc..0351dcef2fce 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index 24e3fd9c5ec7..b9acecf1d8fc 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-twitter-functions diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index 43a55184179f..d6e9225e25ed 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index d05eb79cffa2..f1e254868041 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 690f1d48425c..599f331b0e01 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.54 + 0.210-tw-0.55 twitter-eventlistener-plugin @@ -19,7 +19,7 @@ com.facebook.presto presto-spi - 0.210-tw-0.54 + 0.210-tw-0.55 provided From d52fdaf54be3b8521fd23039b5fcdfde949d916b Mon Sep 17 00:00:00 2001 From: Hao Luo Date: Tue, 25 Sep 2018 11:07:25 -0700 Subject: [PATCH 305/331] update aircompressor version to 0.12 --- .travis.yml | 4 ---- pom.xml | 2 +- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/.travis.yml b/.travis.yml index bf099dd368bd..2bee54ddbb49 100644 --- a/.travis.yml +++ b/.travis.yml @@ -50,10 +50,6 @@ before_install: if [[ ! -e /usr/local/bin/thrift ]]; then sudo ln -s $HOME/.thrift/bin/thrift /usr/local/bin/thrift fi - - | - git clone https://github.com/airlift/aircompressor.git - cd aircompressor && mvn clean install -DskipTests - cd .. install: - ./mvnw -v diff --git a/pom.xml b/pom.xml index 91a6700b18d3..51b3c5260d58 100644 --- a/pom.xml +++ b/pom.xml @@ -419,7 +419,7 @@ io.airlift aircompressor - 0.12-SNAPSHOT + 0.12 From 91f8c95e7878ff612c9d7f10ff1bb70e93866d36 Mon Sep 17 00:00:00 2001 From: Hao Luo Date: Tue, 25 Sep 2018 11:23:36 -0700 Subject: [PATCH 306/331] Revert "Updating travis file to redistribute test load" This reverts commit 21b6a89c8b5b0e08d7a8f594e56e47ddf2e0ed2b. --- .travis.yml | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/.travis.yml b/.travis.yml index 2bee54ddbb49..1054d8d85a50 100644 --- a/.travis.yml +++ b/.travis.yml @@ -21,7 +21,6 @@ env: - PRODUCT_TESTS_BASIC_ENVIRONMENT=true - PRODUCT_TESTS_SPECIFIC_ENVIRONMENT=true - PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_2=true - - PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_3=true - HIVE_TESTS=true - KUDU_TESTS=true @@ -62,7 +61,7 @@ install: ./mvnw install $MAVEN_FAST_INSTALL -P !twitter-modules -pl '!presto-docs,!presto-server,!presto-server-rpm' fi - | - if [[ -v PRODUCT_TESTS_BASIC_ENVIRONMENT || -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT || -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_2 || -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_3 ]]; then + if [[ -v PRODUCT_TESTS_BASIC_ENVIRONMENT || -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT || -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_2 ]]; then ./mvnw install $MAVEN_FAST_INSTALL -P !twitter-modules -pl '!presto-docs,!presto-server-rpm' fi - | @@ -129,7 +128,7 @@ script: singlenode-kerberos-hdfs-impersonation -g storage_formats,cli,hdfs_impersonation,authorization,hive_file_header fi - | - if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_3 ]]; then + if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then presto-product-tests/bin/run_on_docker.sh \ singlenode-kerberos-hdfs-impersonation-cross-realm -g storage_formats,cli,hdfs_impersonation fi From ddaeefdc58407e0eafa3a4df31e78c8ab587f824 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 25 Sep 2018 00:09:17 -0700 Subject: [PATCH 307/331] Copy the original SpnegoAuthentication --- pom.xml | 4 +- twitter-http-client/pom.xml | 179 +++++++++++ .../client/spnego/SpnegoAuthentication.java | 285 ++++++++++++++++++ 3 files changed, 467 insertions(+), 1 deletion(-) create mode 100644 twitter-http-client/pom.xml create mode 100644 twitter-http-client/src/main/java/io/airlift/http/client/spnego/SpnegoAuthentication.java diff --git a/pom.xml b/pom.xml index 51b3c5260d58..1bfc14336bc5 100644 --- a/pom.xml +++ b/pom.xml @@ -45,6 +45,7 @@ 4.7.1 0.172 + 0.172-tw-0.1 ${dep.airlift.version} 0.36 1.11.293 @@ -121,6 +122,7 @@ presto-memory-context presto-proxy presto-kudu + twitter-http-client @@ -539,7 +541,7 @@ io.airlift http-client - ${dep.airlift.version} + ${dep.airlift.twitter.version} diff --git a/twitter-http-client/pom.xml b/twitter-http-client/pom.xml new file mode 100644 index 000000000000..a5ca8bd77489 --- /dev/null +++ b/twitter-http-client/pom.xml @@ -0,0 +1,179 @@ + + + + presto-root + com.facebook.presto + 0.210-tw-0.55 + + 4.0.0 + + io.airlift + http-client + 0.172-tw-0.1 + + + ${project.parent.basedir} + 9.4.12.RC0 + + + + + ch.qos.logback + logback-core + + + + org.eclipse.jetty + jetty-client + ${dep.jetty.version} + + + + org.eclipse.jetty + jetty-io + ${dep.jetty.version} + + + + org.eclipse.jetty + jetty-util + ${dep.jetty.version} + + + + org.eclipse.jetty + jetty-http + ${dep.jetty.version} + + + + org.eclipse.jetty.http2 + http2-client + ${dep.jetty.version} + + + + org.eclipse.jetty.http2 + http2-http-client-transport + ${dep.jetty.version} + + + + io.airlift + concurrent + + + + io.airlift + json + + + + io.airlift + log + + + + io.airlift + units + + + + io.airlift + stats + + + + io.airlift + security + + + + io.airlift + configuration + + + + io.airlift + trace-token + + + + javax.validation + validation-api + + + + javax.inject + javax.inject + + + + org.weakref + jmxutils + + + + com.google.inject + guice + + + + com.google.guava + guava + + + + javax.annotation + javax.annotation-api + + + + com.google.code.findbugs + jsr305 + true + + + + + + org.gaul + modernizer-maven-plugin + + + io.airlift.http.client + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + unpack + generate-sources + + unpack + + + false + + + io.airlift + http-client + ${dep.airlift.version} + jar + true + ${project.build.directory}/classes + **/SpnegoAuthentication.class + + + + + + + + + diff --git a/twitter-http-client/src/main/java/io/airlift/http/client/spnego/SpnegoAuthentication.java b/twitter-http-client/src/main/java/io/airlift/http/client/spnego/SpnegoAuthentication.java new file mode 100644 index 000000000000..60c3e40b2d08 --- /dev/null +++ b/twitter-http-client/src/main/java/io/airlift/http/client/spnego/SpnegoAuthentication.java @@ -0,0 +1,285 @@ +package io.airlift.http.client.spnego; + +import com.google.common.collect.ImmutableMap; +import com.sun.security.auth.module.Krb5LoginModule; +import io.airlift.log.Logger; +import io.airlift.units.Duration; +import org.eclipse.jetty.client.api.Authentication; +import org.eclipse.jetty.client.api.ContentResponse; +import org.eclipse.jetty.client.api.Request; +import org.eclipse.jetty.http.HttpHeader; +import org.eclipse.jetty.util.Attributes; +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.UncheckedIOException; +import java.net.InetAddress; +import java.net.URI; +import java.net.UnknownHostException; +import java.security.Principal; +import java.security.PrivilegedAction; +import java.util.Base64; +import java.util.Locale; +import java.util.concurrent.TimeUnit; + +import static com.google.common.base.Preconditions.checkState; +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; +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; + +public class SpnegoAuthentication + implements Authentication +{ + private static final String NEGOTIATE = HttpHeader.NEGOTIATE.asString(); + private static final Logger LOG = Logger.get(SpnegoAuthentication.class); + private static final Duration MIN_CREDENTIAL_LIFE_TIME = new Duration(60, TimeUnit.SECONDS); + + private static final GSSManager GSS_MANAGER = GSSManager.getInstance(); + + private static final Oid SPNEGO_OID; + private static final Oid KERBEROS_OID; + + static { + try { + SPNEGO_OID = new Oid("1.3.6.1.5.5.2"); + KERBEROS_OID = new Oid("1.2.840.113554.1.2.2"); + } + catch (GSSException e) { + throw new AssertionError(e); + } + } + + private final File keytab; + private final File credentialCache; + private final String principal; + private final String remoteServiceName; + private final boolean useCanonicalHostname; + + @GuardedBy("this") + private Session clientSession; + + public SpnegoAuthentication(File keytab, File kerberosConfig, File credentialCache, String principal, String remoteServiceName, boolean useCanonicalHostname) + { + requireNonNull(kerberosConfig, "Kerberos config path is null"); + requireNonNull(remoteServiceName, "Kerberos remote service name is null"); + + this.keytab = keytab; + this.credentialCache = credentialCache; + this.principal = principal; + this.remoteServiceName = remoteServiceName; + this.useCanonicalHostname = useCanonicalHostname; + + System.setProperty("java.security.krb5.conf", kerberosConfig.getAbsolutePath()); + } + + @Override + public Result authenticate(Request request, ContentResponse response, HeaderInfo headerInfo, Attributes attributes) + { + URI normalizedUri = UriUtil.normalizedUri(request.getURI()); + + return new Result() + { + @Override + public URI getURI() + { + return normalizedUri; + } + + @Override + public void apply(Request request) + { + GSSContext context = null; + try { + String servicePrincipal = makeServicePrincipal(remoteServiceName, normalizedUri.getHost(), useCanonicalHostname); + 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) { + request.header(headerInfo.getHeader(), format("%s %s", NEGOTIATE, Base64.getEncoder().encodeToString(token))); + } + else { + throw new RuntimeException(format("No token generated from GSS context for %s", request.getURI())); + } + } + catch (GSSException e) { + throw new RuntimeException(format("Failed to establish GSSContext for request %s", request.getURI()), e); + } + catch (LoginException e) { + throw new RuntimeException(format("Failed to establish LoginContext for request %s", request.getURI()), e); + } + finally { + try { + if (context != null) { + context.dispose(); + } + } + catch (GSSException e) { + // ignore + } + } + } + }; + } + + @Override + public boolean matches(String type, URI uri, String realm) + { + // The class matches all requests for Negotiate scheme. Realm is not used for now + return NEGOTIATE.equalsIgnoreCase(type); + } + + private synchronized Session getSession() + throws LoginException, GSSException + { + if (clientSession == null || clientSession.getClientCredential().getRemainingLifetime() < MIN_CREDENTIAL_LIFE_TIME.getValue(TimeUnit.SECONDS)) { + // 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 optionsBuilder = ImmutableMap.builder(); + optionsBuilder.put("refreshKrb5Config", "true"); + optionsBuilder.put("doNotPrompt", "true"); + optionsBuilder.put("useKeyTab", "true"); + if (LOG.isDebugEnabled()) { + optionsBuilder.put("debug", "true"); + } + + if (keytab != null) { + optionsBuilder.put("keyTab", keytab.getAbsolutePath()); + } + + if (credentialCache != null) { + optionsBuilder.put("ticketCache", credentialCache.getAbsolutePath()); + optionsBuilder.put("useTicketCache", "true"); + optionsBuilder.put("renewTGT", "true"); + } + + if (principal != null) { + optionsBuilder.put("principal", principal); + } + + return new AppConfigurationEntry[] { + new AppConfigurationEntry(Krb5LoginModule.class.getName(), REQUIRED, optionsBuilder.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)); + + clientSession = new Session(loginContext, clientCredential); + } + + return clientSession; + } + + 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(); + } + checkState(!fullHostName.equalsIgnoreCase("localhost"), "Fully qualified name of localhost should not resolve to 'localhost'. System configuration error?"); + return fullHostName; + } + catch (UnknownHostException e) { + throw new UncheckedIOException(e); + } + } + + private interface GssSupplier + { + T get() + throws GSSException; + } + + private static T doAs(Subject subject, GssSupplier action) + { + return Subject.doAs(subject, (PrivilegedAction) () -> { + try { + return action.get(); + } + catch (GSSException e) { + throw new RuntimeException(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; + } + } +} From 4155a117892fdfbd4925ef4f4989cc813a59ab18 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 25 Sep 2018 18:40:18 -0700 Subject: [PATCH 308/331] Modify airlift http client to support user name based service principal --- .../client/spnego/SpnegoAuthentication.java | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/twitter-http-client/src/main/java/io/airlift/http/client/spnego/SpnegoAuthentication.java b/twitter-http-client/src/main/java/io/airlift/http/client/spnego/SpnegoAuthentication.java index 60c3e40b2d08..d4f96ccc82ef 100644 --- a/twitter-http-client/src/main/java/io/airlift/http/client/spnego/SpnegoAuthentication.java +++ b/twitter-http-client/src/main/java/io/airlift/http/client/spnego/SpnegoAuthentication.java @@ -1,3 +1,16 @@ +/* + * 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 io.airlift.http.client.spnego; import com.google.common.collect.ImmutableMap; @@ -69,6 +82,7 @@ public class SpnegoAuthentication private final File credentialCache; private final String principal; private final String remoteServiceName; + private final boolean isCompleteServicePrinciple; private final boolean useCanonicalHostname; @GuardedBy("this") @@ -83,6 +97,7 @@ public SpnegoAuthentication(File keytab, File kerberosConfig, File credentialCac this.credentialCache = credentialCache; this.principal = principal; this.remoteServiceName = remoteServiceName; + this.isCompleteServicePrinciple = remoteServiceName.contains("@"); this.useCanonicalHostname = useCanonicalHostname; System.setProperty("java.security.krb5.conf", kerberosConfig.getAbsolutePath()); @@ -106,11 +121,11 @@ public void apply(Request request) { GSSContext context = null; try { - String servicePrincipal = makeServicePrincipal(remoteServiceName, normalizedUri.getHost(), useCanonicalHostname); + String servicePrincipal = isCompleteServicePrinciple ? remoteServiceName : makeServicePrincipal(remoteServiceName, normalizedUri.getHost(), useCanonicalHostname); Session session = getSession(); context = doAs(session.getLoginContext().getSubject(), () -> { GSSContext result = GSS_MANAGER.createContext( - GSS_MANAGER.createName(servicePrincipal, NT_HOSTBASED_SERVICE), + GSS_MANAGER.createName(servicePrincipal, isCompleteServicePrinciple ? NT_USER_NAME : NT_HOSTBASED_SERVICE), SPNEGO_OID, session.getClientCredential(), INDEFINITE_LIFETIME); From f342e04bde125a9aa1e7f07c9c1f0ce007ca7cdd Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 26 Oct 2018 22:39:53 -0700 Subject: [PATCH 309/331] Support event based Slack notification (#177) * Add Slack bot * Support file based notification template and failure treatment template --- twitter-eventlistener-plugin/pom.xml | 68 +++- .../eventlistener/TwitterEventHandler.java | 33 ++ .../eventlistener/TwitterEventListener.java | 24 +- .../TwitterEventListenerConfig.java | 127 ++++++ .../TwitterEventListenerFactory.java | 24 +- .../TwitterEventListenerModule.java | 43 ++ .../knowledge/KnowledgeBase.java | 55 +++ .../knowledge/KnowledgeBases.java | 47 +++ .../QueryCompletedEventScriber.java | 21 +- .../{ => scriber}/QueryStatsHelper.java | 2 +- .../{ => scriber}/TwitterScriber.java | 2 +- .../plugin/eventlistener/slack/SlackBot.java | 378 ++++++++++++++++++ .../slack/SlackBotCredentials.java | 55 +++ .../eventlistener/slack/SlackChannel.java | 35 ++ .../slack/SlackChatPostMessageRequest.java | 46 +++ .../slack/SlackChatPostMessageResponse.java | 41 ++ .../slack/SlackImHistoryRequest.java | 46 +++ .../slack/SlackImHistoryResponse.java | 60 +++ .../slack/SlackImOpenRequest.java | 35 ++ .../slack/SlackImOpenResponse.java | 41 ++ .../eventlistener/slack/SlackMessage.java | 44 ++ .../slack/SlackNotificationTemplate.java | 51 +++ .../slack/SlackNotificationTemplates.java | 47 +++ .../eventlistener/slack/SlackResponse.java | 42 ++ .../plugin/eventlistener/slack/SlackUser.java | 44 ++ .../SlackUsersLookupByEmailResponse.java | 41 ++ .../TestTwitterEventListenerConfig.java | 66 +++ .../TestTwitterEventListenerPlugin.java | 56 +++ 28 files changed, 1565 insertions(+), 9 deletions(-) create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventHandler.java create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerConfig.java create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerModule.java create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/knowledge/KnowledgeBase.java create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/knowledge/KnowledgeBases.java rename twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/{ => scriber}/QueryCompletedEventScriber.java (89%) rename twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/{ => scriber}/QueryStatsHelper.java (99%) rename twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/{ => scriber}/TwitterScriber.java (97%) create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBot.java create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBotCredentials.java create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackChannel.java create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackChatPostMessageRequest.java create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackChatPostMessageResponse.java create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackImHistoryRequest.java create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackImHistoryResponse.java create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackImOpenRequest.java create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackImOpenResponse.java create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackMessage.java create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplate.java create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplates.java create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackResponse.java create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackUser.java create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackUsersLookupByEmailResponse.java create mode 100644 twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/TestTwitterEventListenerConfig.java create mode 100644 twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/TestTwitterEventListenerPlugin.java diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 599f331b0e01..c0892e67e5e7 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -19,22 +19,72 @@ com.facebook.presto presto-spi - 0.210-tw-0.55 provided + + + io.airlift + bootstrap + + + + io.airlift + configuration + + + + io.airlift + json + + + com.fasterxml.jackson.core + jackson-annotations + + + + io.airlift log + io.airlift units provided + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + + com.fasterxml.jackson.core + jackson-databind + + com.google.guava guava + + + com.google.inject + guice + + + + com.squareup.okhttp3 + okhttp + + + + javax.inject + javax.inject + + org.glassfish javax.json @@ -85,6 +135,7 @@ + com.twitter util-logging_2.10 @@ -96,10 +147,12 @@ + org.apache.thrift libthrift + org.scala-lang scala-library @@ -111,5 +164,18 @@ + + + + org.testng + testng + test + + + + io.airlift + testing + test + diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventHandler.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventHandler.java new file mode 100644 index 000000000000..36f44f68deaa --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventHandler.java @@ -0,0 +1,33 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.twitter.presto.plugin.eventlistener; + +import com.facebook.presto.spi.eventlistener.QueryCompletedEvent; +import com.facebook.presto.spi.eventlistener.QueryCreatedEvent; +import com.facebook.presto.spi.eventlistener.SplitCompletedEvent; + +public interface TwitterEventHandler +{ + default void handleQueryCreated(QueryCreatedEvent queryCreatedEvent) + { + } + + default void handleQueryCompleted(QueryCompletedEvent queryCompletedEvent) + { + } + + default void handleSplitCompleted(SplitCompletedEvent splitCompletedEvent) + { + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java index 6df4eb29c54b..45339f953064 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListener.java @@ -19,25 +19,45 @@ import com.facebook.presto.spi.eventlistener.SplitCompletedEvent; import io.airlift.log.Logger; +import javax.inject.Inject; + +import java.util.Set; + +import static java.util.Objects.requireNonNull; + public class TwitterEventListener implements EventListener { private static final Logger log = Logger.get(TwitterEventListener.class); - private final QueryCompletedEventScriber scriber = new QueryCompletedEventScriber(); + private final Set handlers; + + @Inject + public TwitterEventListener(Set handlers) + { + this.handlers = requireNonNull(handlers, "handlers is null"); + } @Override public void queryCreated(QueryCreatedEvent queryCreatedEvent) { + for (TwitterEventHandler handler : handlers) { + handler.handleQueryCreated(queryCreatedEvent); + } } @Override public void queryCompleted(QueryCompletedEvent queryCompletedEvent) { - scriber.handle(queryCompletedEvent); + for (TwitterEventHandler handler : handlers) { + handler.handleQueryCompleted(queryCompletedEvent); + } } @Override public void splitCompleted(SplitCompletedEvent splitCompletedEvent) { + for (TwitterEventHandler handler : handlers) { + handler.handleSplitCompleted(splitCompletedEvent); + } } } diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerConfig.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerConfig.java new file mode 100644 index 000000000000..0efdcbe2f3f0 --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerConfig.java @@ -0,0 +1,127 @@ +/* + * 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.twitter.presto.plugin.eventlistener; + +import com.google.common.net.HostAndPort; +import io.airlift.configuration.Config; + +import java.net.URI; + +public class TwitterEventListenerConfig +{ + private String slackConfigFile; + private String slackUsers; + private HostAndPort slackHttpProxy; + private URI slackUri; + private String slackEmailTemplate; + private String slackNotificationTemplateFile; + private String knowledgeBaseFile; + private String scribeCategory; + + public String getSlackConfigFile() + { + return slackConfigFile; + } + + @Config("event-listener.slack-config-file") + public TwitterEventListenerConfig setSlackConfigFile(String slackConfigFile) + { + this.slackConfigFile = slackConfigFile; + return this; + } + + public String getSlackUsers() + { + return slackUsers; + } + + @Config("event-listener.slack-users") + public TwitterEventListenerConfig setSlackUsers(String slackUsers) + { + this.slackUsers = slackUsers; + return this; + } + + public HostAndPort getSlackHttpProxy() + { + return slackHttpProxy; + } + + @Config("event-listener.slack-http-proxy") + public TwitterEventListenerConfig setSlackHttpProxy(HostAndPort slackHttpProxy) + { + this.slackHttpProxy = slackHttpProxy; + return this; + } + + public URI getSlackUri() + { + return slackUri; + } + + @Config("event-listener.slack-uri") + public TwitterEventListenerConfig setSlackUri(URI slackUri) + { + this.slackUri = slackUri; + return this; + } + + public String getSlackEmailTemplate() + { + return slackEmailTemplate; + } + + @Config("event-listener.slack-email-template") + public TwitterEventListenerConfig setSlackEmailTemplate(String slackEmailTemplate) + { + this.slackEmailTemplate = slackEmailTemplate; + return this; + } + + public String getSlackNotificationTemplateFile() + { + return slackNotificationTemplateFile; + } + + @Config("event-listener.slack-notification-template-file") + public TwitterEventListenerConfig setSlackNotificationTemplateFile(String slackNotificationTemplateFile) + { + this.slackNotificationTemplateFile = slackNotificationTemplateFile; + return this; + } + + public String getKnowledgeBaseFile() + { + return knowledgeBaseFile; + } + + @Config("event-listener.knowledge-base-file") + public TwitterEventListenerConfig setKnowledgeBaseFile(String knowledgeBaseFile) + { + this.knowledgeBaseFile = knowledgeBaseFile; + return this; + } + + public String getScribeCategory() + { + return scribeCategory; + } + + @Config("event-listener.scribe-category") + public TwitterEventListenerConfig setScribeCategory(String scribeCategory) + { + this.scribeCategory = scribeCategory; + return this; + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerFactory.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerFactory.java index a22762c9ec32..c28567b47c82 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerFactory.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerFactory.java @@ -15,9 +15,13 @@ import com.facebook.presto.spi.eventlistener.EventListener; import com.facebook.presto.spi.eventlistener.EventListenerFactory; +import com.google.inject.Injector; +import io.airlift.bootstrap.Bootstrap; import java.util.Map; +import static com.google.common.base.Throwables.throwIfUnchecked; + public class TwitterEventListenerFactory implements EventListenerFactory { @@ -30,6 +34,24 @@ public String getName() @Override public EventListener create(Map config) { - return new TwitterEventListener(); + try { + Bootstrap app = new Bootstrap(new TwitterEventListenerModule()); + + Injector injector = app + .strictConfig() + .doNotInitializeLogging() + .setRequiredConfigurationProperties(config) + .initialize(); + + return injector.getInstance(TwitterEventListener.class); + } + catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted while creating connector", ie); + } + catch (Exception e) { + throwIfUnchecked(e); + throw new RuntimeException(e); + } } } diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerModule.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerModule.java new file mode 100644 index 000000000000..65c68b597fba --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterEventListenerModule.java @@ -0,0 +1,43 @@ +/* + * 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.twitter.presto.plugin.eventlistener; + +import com.google.inject.Binder; +import com.google.inject.Scopes; +import com.google.inject.multibindings.Multibinder; +import com.twitter.presto.plugin.eventlistener.scriber.QueryCompletedEventScriber; +import com.twitter.presto.plugin.eventlistener.slack.SlackBot; +import io.airlift.configuration.AbstractConfigurationAwareModule; + +import static com.google.inject.multibindings.Multibinder.newSetBinder; +import static io.airlift.configuration.ConfigBinder.configBinder; + +public class TwitterEventListenerModule + extends AbstractConfigurationAwareModule +{ + @Override + public void setup(Binder binder) + { + binder.bind(TwitterEventListener.class).in(Scopes.SINGLETON); + configBinder(binder).bindConfig(TwitterEventListenerConfig.class); + TwitterEventListenerConfig config = buildConfigObject(TwitterEventListenerConfig.class); + Multibinder twitterEventHandlerBinder = newSetBinder(binder, TwitterEventHandler.class); + if (config.getScribeCategory() != null) { + twitterEventHandlerBinder.addBinding().to(QueryCompletedEventScriber.class).in(Scopes.SINGLETON); + } + if (config.getSlackConfigFile() != null) { + twitterEventHandlerBinder.addBinding().to(SlackBot.class).in(Scopes.SINGLETON); + } + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/knowledge/KnowledgeBase.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/knowledge/KnowledgeBase.java new file mode 100644 index 000000000000..4bf4043446e1 --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/knowledge/KnowledgeBase.java @@ -0,0 +1,55 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.twitter.presto.plugin.eventlistener.knowledge; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Optional; +import java.util.regex.Pattern; + +public class KnowledgeBase +{ + private final Pattern failureMessageRegex; + private final String treatment; + + @JsonCreator + public KnowledgeBase( + @JsonProperty("failure_message") Pattern failureMessageRegex, + @JsonProperty("treatment") String treatment) + { + this.failureMessageRegex = failureMessageRegex; + this.treatment = treatment; + } + + @JsonProperty + public Pattern getFailureMessageRegex() + { + return failureMessageRegex; + } + + @JsonProperty + public String getTreatment() + { + return treatment; + } + + public Optional match(String failureMessage) + { + if (failureMessageRegex.matcher(failureMessage).matches()) { + return Optional.of(treatment); + } + return Optional.empty(); + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/knowledge/KnowledgeBases.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/knowledge/KnowledgeBases.java new file mode 100644 index 000000000000..2090f9568869 --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/knowledge/KnowledgeBases.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.twitter.presto.plugin.eventlistener.knowledge; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; +import java.util.Optional; + +public class KnowledgeBases +{ + private final List knowledge; + + @JsonCreator + public KnowledgeBases( + @JsonProperty("knowledge") List knowledge) + { + this.knowledge = knowledge; + } + + @JsonProperty + public List getKnowledge() + { + return knowledge; + } + + public Optional getTreatment(String failureMessage) + { + return knowledge.stream() + .map(knowledge -> knowledge.match(failureMessage)) + .filter(Optional::isPresent) + .map(Optional::get) + .findFirst(); + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/scriber/QueryCompletedEventScriber.java similarity index 89% rename from twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java rename to twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/scriber/QueryCompletedEventScriber.java index 6e0d31fd3352..afd2230ec3c1 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryCompletedEventScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/scriber/QueryCompletedEventScriber.java @@ -11,33 +11,48 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.twitter.presto.plugin.eventlistener; +package com.twitter.presto.plugin.eventlistener.scriber; import com.facebook.presto.spi.eventlistener.QueryCompletedEvent; import com.facebook.presto.spi.eventlistener.QueryContext; import com.facebook.presto.spi.eventlistener.QueryFailureInfo; import com.facebook.presto.spi.eventlistener.QueryMetadata; import com.facebook.presto.spi.eventlistener.QueryStatistics; +import com.twitter.presto.plugin.eventlistener.TwitterEventHandler; +import com.twitter.presto.plugin.eventlistener.TwitterEventListenerConfig; import com.twitter.presto.thriftjava.QueryCompletionEvent; import com.twitter.presto.thriftjava.QueryState; import io.airlift.log.Logger; import org.apache.thrift.TException; +import javax.inject.Inject; + import java.util.HashMap; import java.util.List; import java.util.Map; +import static java.util.Objects.requireNonNull; + /** * Class that scribes query completion events */ public class QueryCompletedEventScriber + implements TwitterEventHandler { private static final String DASH = "-"; private static final Logger log = Logger.get(QueryCompletedEventScriber.class); - private TwitterScriber scriber = new TwitterScriber("presto_query_completion"); + private final TwitterScriber scriber; + + @Inject + public QueryCompletedEventScriber(TwitterEventListenerConfig config) + { + requireNonNull(config.getScribeCategory(), "scribe category is null"); + this.scriber = new TwitterScriber(config.getScribeCategory()); + } - public void handle(QueryCompletedEvent event) + @Override + public void handleQueryCompleted(QueryCompletedEvent event) { try { scriber.scribe(toThriftQueryCompletionEvent(event)); diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/scriber/QueryStatsHelper.java similarity index 99% rename from twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java rename to twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/scriber/QueryStatsHelper.java index 7f5456dce2ae..6d24d888c75d 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/QueryStatsHelper.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/scriber/QueryStatsHelper.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.twitter.presto.plugin.eventlistener; +package com.twitter.presto.plugin.eventlistener.scriber; import com.facebook.presto.spi.eventlistener.QueryMetadata; import com.facebook.presto.spi.eventlistener.QueryStatistics; diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/scriber/TwitterScriber.java similarity index 97% rename from twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java rename to twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/scriber/TwitterScriber.java index 8f4088da6aeb..225d0c16d25e 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/TwitterScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/scriber/TwitterScriber.java @@ -11,7 +11,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.twitter.presto.plugin.eventlistener; +package com.twitter.presto.plugin.eventlistener.scriber; import com.twitter.logging.BareFormatter$; import com.twitter.logging.Level; diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBot.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBot.java new file mode 100644 index 000000000000..81359f69fb0e --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBot.java @@ -0,0 +1,378 @@ +/* + * 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.twitter.presto.plugin.eventlistener.slack; + +import com.facebook.presto.spi.eventlistener.QueryCompletedEvent; +import com.facebook.presto.spi.eventlistener.QueryCreatedEvent; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.net.HostAndPort; +import com.twitter.presto.plugin.eventlistener.TwitterEventHandler; +import com.twitter.presto.plugin.eventlistener.TwitterEventListenerConfig; +import com.twitter.presto.plugin.eventlistener.knowledge.KnowledgeBases; +import io.airlift.json.ObjectMapperProvider; +import io.airlift.log.Logger; +import okhttp3.Authenticator; +import okhttp3.Call; +import okhttp3.Callback; +import okhttp3.Credentials; +import okhttp3.FormBody; +import okhttp3.HttpUrl; +import okhttp3.MediaType; +import okhttp3.OkHttpClient; +import okhttp3.Request; +import okhttp3.RequestBody; +import okhttp3.Response; + +import javax.inject.Inject; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.Proxy; +import java.net.URI; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.Optional; +import java.util.function.Consumer; +import java.util.regex.Pattern; + +import static com.google.common.net.HttpHeaders.AUTHORIZATION; +import static com.google.common.net.HttpHeaders.CONTENT_TYPE; +import static com.google.common.net.HttpHeaders.PROXY_AUTHORIZATION; +import static java.lang.String.format; +import static java.net.Proxy.Type.HTTP; +import static java.nio.charset.StandardCharsets.UTF_8; +import static java.util.Objects.requireNonNull; + +public class SlackBot + implements TwitterEventHandler +{ + private static final MediaType JSON_CONTENT_TYPE = MediaType.parse("Content-type: application/json; charset=utf-8"); + private static final String USER = "\\$\\{USER}"; + private static final String QUERY_ID = "\\$\\{QUERY_ID}"; + private static final String PRINCIPAL = "\\$\\{PRINCIPAL}"; + private static final String STATE = "\\$\\{STATE}"; + private static final String FAILURE_MESSAGE = "\\$\\{FAILURE_MESSAGE}"; + private static final String FAILURE_TREATMENT = "\\$\\{FAILURE_TREATMENT}"; + private static final String DASH = "-"; + private static final String CREATED = "created"; + private static final String COMPLETED = "completed"; + private static final String STOP = "stop"; + private static final String STOP_PRINCIPAL = "stop principal=%s"; + private static final String STOP_EVENT = "stop event=%s"; + private static final String STOP_STATE = "stop state=%s"; + private static final String RESUME = "resume"; + private static final String RESUME_PRINCIPAL = "resume principal=%s"; + private static final String RESUME_EVENT = "resume event=%s"; + private static final String RESUME_STATE = "resume state=%s"; + private final Logger log = Logger.get(SlackBot.class); + private final SlackBotCredentials slackBotCredentials; + private final Pattern slackUsers; + private final URI slackUri; + private final String emailTemplate; + private final SlackNotificationTemplates notificationTemplates; + private final Optional knowledgeBases; + private final OkHttpClient client; + + @Inject + public SlackBot(TwitterEventListenerConfig config) + throws IOException + { + requireNonNull(config.getSlackConfigFile(), "slack config file is null"); + this.slackBotCredentials = parse(Files.readAllBytes(Paths.get(config.getSlackConfigFile())), SlackBotCredentials.class); + this.slackUsers = Pattern.compile(requireNonNull(config.getSlackUsers())); + this.slackUri = requireNonNull(config.getSlackUri()); + this.emailTemplate = requireNonNull(config.getSlackEmailTemplate()); + this.notificationTemplates = parse(Files.readAllBytes(Paths.get(config.getSlackNotificationTemplateFile())), SlackNotificationTemplates.class); + if (config.getKnowledgeBaseFile() != null) { + this.knowledgeBases = Optional.of(parse(Files.readAllBytes(Paths.get(config.getKnowledgeBaseFile())), KnowledgeBases.class)); + } + else { + this.knowledgeBases = Optional.empty(); + } + + OkHttpClient.Builder builder = new OkHttpClient.Builder(); + + if (slackBotCredentials.getProxyUser().isPresent() && slackBotCredentials.getProxyPassword().isPresent() && config.getSlackHttpProxy() != null) { + setupHttpProxy(builder, config.getSlackHttpProxy()); + builder.proxyAuthenticator(basicAuth(PROXY_AUTHORIZATION, slackBotCredentials.getProxyUser().get(), slackBotCredentials.getProxyPassword().get())); + } + + this.client = builder.build(); + } + + @Override + public void handleQueryCreated(QueryCreatedEvent queryCreatedEvent) + { + handleSlackNotification(CREATED, + queryCreatedEvent.getContext().getUser(), + queryCreatedEvent.getMetadata().getQueryId(), + queryCreatedEvent.getContext().getPrincipal(), + queryCreatedEvent.getMetadata().getQueryState(), + Optional.empty()); + } + + @Override + public void handleQueryCompleted(QueryCompletedEvent queryCompletedEvent) + { + handleSlackNotification(COMPLETED, + queryCompletedEvent.getContext().getUser(), + queryCompletedEvent.getMetadata().getQueryId(), + queryCompletedEvent.getContext().getPrincipal(), + queryCompletedEvent.getMetadata().getQueryState(), + queryCompletedEvent.getFailureInfo().map(queryFailureInfo -> queryFailureInfo.getFailureMessage().orElse("unknown"))); + } + + private void handleSlackNotification(String event, String user, String queryId, Optional principal, String state, Optional failureMessage) + { + if (!slackUsers.matcher(user).matches()) { + return; + } + Optional template = notificationTemplates.getText(user, event, state); + if (!template.isPresent()) { + return; + } + Optional treatment = failureMessage.map(message -> knowledgeBases.map(knowledge -> knowledge.getTreatment(message).orElse(DASH)).orElse(DASH)); + try { + String email = emailTemplate.replaceAll(USER, user); + String text = template.get() + .replaceAll(QUERY_ID, queryId) + .replaceAll(STATE, state) + .replaceAll(PRINCIPAL, principal.orElse(DASH)) + .replaceAll(FAILURE_MESSAGE, failureMessage.orElse(DASH)) + .replaceAll(FAILURE_TREATMENT, treatment.orElse(DASH)); + Consumer sender = userLookupByEmail(openChannel(slackImOpenResponse -> { + shouldSend(slackImOpenResponse, Optional.empty(), event, principal, state, postMessage(text, slackChatPostMessageResponse -> { + log.debug(format("sent the following message to user %s:\n%s\n", user, slackChatPostMessageResponse.getMessage().map(SlackMessage::getText).orElse("unknown"))); + })); + })); + sender.accept(email); + } + catch (Exception e) { + log.warn(e, "Failed to send the slack notification"); + } + } + + private Consumer userLookupByEmail(Consumer next) + { + return email -> { + FormBody body = new FormBody.Builder(UTF_8) + .add("email", email) + .build(); + postForm("/api/users.lookupByEmail", + body, + SlackUsersLookupByEmailResponse.class, + next); + }; + } + + private Consumer openChannel(Consumer next) + { + return slackUsersLookupByEmailResponse -> { + String userId = slackUsersLookupByEmailResponse.getUser().orElseThrow(() -> new RuntimeException("Failed to get user info")).getId(); + postJson("/api/im.open", + encode(new SlackImOpenRequest(userId), SlackImOpenRequest.class), + SlackImOpenResponse.class, + next); + }; + } + + private void shouldSend(SlackImOpenResponse response, Optional latest, String event, Optional principal, String state, Consumer postMessage) + { + SlackChannel channel = response.getChannel().orElseThrow(() -> new RuntimeException("Failed to open the user channel")); + Consumer> checker = getChannelHistory(channel.getId(), history -> { + Optional newLatest = latest; + if (!history.getMessages().isPresent()) { + postMessage.accept(response); + return; + } + for (SlackMessage message : history.getMessages().get()) { + Optional result = shouldSend(message, event, principal, state); + if (result.isPresent()) { + if (result.get()) { + postMessage.accept(response); + } + return; + } + if (!newLatest.isPresent() || Double.valueOf(newLatest.get()) > Double.valueOf(message.getTs())) { + newLatest = Optional.of(message.getTs()); + } + } + if (!history.getHasMore().isPresent() || !history.getHasMore().get()) { + postMessage.accept(response); + return; + } + shouldSend(response, newLatest, event, principal, state, postMessage); + }); + checker.accept(latest); + } + + private Optional shouldSend(SlackMessage message, String event, Optional principal, String state) + { + String text = message.getText().trim(); + if (message.getText().trim().equalsIgnoreCase(RESUME)) { + return Optional.of(true); + } + if (principal.isPresent() && text.equalsIgnoreCase(format(RESUME_PRINCIPAL, principal.get()))) { + return Optional.of(true); + } + if (text.equalsIgnoreCase(format(RESUME_EVENT, event))) { + return Optional.of(true); + } + if (text.equalsIgnoreCase(format(RESUME_STATE, state))) { + return Optional.of(true); + } + if (text.equalsIgnoreCase(STOP)) { + return Optional.of(false); + } + if (principal.isPresent() && text.equalsIgnoreCase(format(STOP_PRINCIPAL, principal.get()))) { + return Optional.of(false); + } + if (text.equalsIgnoreCase(format(STOP_EVENT, event))) { + return Optional.of(false); + } + if (text.equalsIgnoreCase(format(STOP_STATE, state))) { + return Optional.of(false); + } + + return Optional.empty(); + } + + private Consumer> getChannelHistory(String channel, Consumer next) + { + return latest -> { + FormBody.Builder body = new FormBody.Builder(UTF_8) + .add("channel", channel); + latest.ifPresent(ts -> body.add("latest", ts)); + postForm("/api/im.history", + body.build(), + SlackImHistoryResponse.class, + next); + }; + } + + private Consumer postMessage(String text, Consumer next) + { + return slackImOpenResponse -> { + String channel = slackImOpenResponse.getChannel().orElseThrow(() -> new RuntimeException("Failed to open the user channel")).getId(); + postJson("/api/chat.postMessage", + encode(new SlackChatPostMessageRequest(channel, text), SlackChatPostMessageRequest.class), + SlackChatPostMessageResponse.class, + next); + }; + } + + private void postForm(String path, R body, Class javaType, Consumer next) + { + String type = "application/x-www-form-urlencoded; charset=utf-8"; + post(path, type, body, javaType, next); + } + + private void postJson(String path, R body, Class javaType, Consumer next) + { + String type = "application/json; charset=utf-8"; + post(path, type, body, javaType, next); + } + + private void post(String path, String type, RequestBody body, Class javaType, Consumer next) + { + HttpUrl url = HttpUrl.get(URI.create(slackUri.toString() + path)); + + Request request = new Request.Builder() + .url(requireNonNull(url)) + .header(CONTENT_TYPE, type) + .header(AUTHORIZATION, "Bearer " + slackBotCredentials.getToken()) + .post(body) + .build(); + client.newCall(request).enqueue(new Callback() + { + @Override + public void onFailure(Call call, IOException e) + { + log.warn(e, "Failed to send the slack notification"); + } + + @Override + public void onResponse(Call call, Response response) + throws IOException + { + requireNonNull(response.body(), "response.body() is null"); + T content = parse(response.body().bytes(), javaType); + if (!content.isOk()) { + throw new RuntimeException(format("Slack responded an error message: %s", content.getError().orElse("unknown"))); + } + next.accept(content); + } + }); + } + + private static void setupHttpProxy(OkHttpClient.Builder clientBuilder, HostAndPort httpProxy) + { + clientBuilder.proxy(new Proxy(HTTP, toUnresolvedAddress(httpProxy))); + } + + private static InetSocketAddress toUnresolvedAddress(HostAndPort address) + { + return InetSocketAddress.createUnresolved(address.getHost(), address.getPort()); + } + + public static Authenticator basicAuth(String scope, String user, String password) + { + requireNonNull(user, "user is null"); + requireNonNull(password, "password is null"); + if (user.contains(":")) { + throw new RuntimeException("Illegal character ':' found in username"); + } + + return createAuthenticator(scope, Credentials.basic(user, password)); + } + + private static Authenticator createAuthenticator(String scope, String credential) + { + return (route, response) -> { + if (response.request().header(scope) != null) { + return null; // Give up, we've already failed to authenticate. + } + + return response.request().newBuilder() + .header(scope, credential) + .build(); + }; + } + + private static T parse(byte[] json, Class javaType) + { + ObjectMapper mapper = new ObjectMapperProvider().get() + .disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES); + try { + return mapper.readValue(json, javaType); + } + catch (IOException e) { + throw new IllegalArgumentException(format("Invalid JSON string for %s", javaType), e); + } + } + + private static RequestBody encode(Object json, Class javaType) + { + ObjectMapper mapper = new ObjectMapperProvider().get() + .disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES); + try { + String data = mapper.writerFor(javaType).writeValueAsString(json); + return RequestBody.create(JSON_CONTENT_TYPE, data); + } + catch (IOException e) { + throw new IllegalArgumentException(format("Invalid JSON string for %s", javaType), e); + } + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBotCredentials.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBotCredentials.java new file mode 100644 index 000000000000..8d10d6ca5bc2 --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBotCredentials.java @@ -0,0 +1,55 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.twitter.presto.plugin.eventlistener.slack; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Optional; + +public class SlackBotCredentials +{ + private final String token; + private final Optional proxyUser; + private final Optional proxyPassword; + + @JsonCreator + public SlackBotCredentials( + @JsonProperty("token") String token, + @JsonProperty("proxyUser") Optional proxyUser, + @JsonProperty("proxyPassword") Optional proxyPassword) + { + this.token = token; + this.proxyUser = proxyUser; + this.proxyPassword = proxyPassword; + } + + @JsonProperty + public String getToken() + { + return token; + } + + @JsonProperty + public Optional getProxyUser() + { + return proxyUser; + } + + @JsonProperty + public Optional getProxyPassword() + { + return proxyPassword; + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackChannel.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackChannel.java new file mode 100644 index 000000000000..29abc7e6221d --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackChannel.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.twitter.presto.plugin.eventlistener.slack; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class SlackChannel +{ + private final String id; + + @JsonCreator + public SlackChannel( + @JsonProperty("id") String id) + { + this.id = id; + } + + @JsonProperty + public String getId() + { + return id; + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackChatPostMessageRequest.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackChatPostMessageRequest.java new file mode 100644 index 000000000000..ebfff9842a8b --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackChatPostMessageRequest.java @@ -0,0 +1,46 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.twitter.presto.plugin.eventlistener.slack; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import static java.util.Objects.requireNonNull; + +public class SlackChatPostMessageRequest +{ + private final String channel; + private final String text; + + @JsonCreator + public SlackChatPostMessageRequest( + @JsonProperty("channel") String channel, + @JsonProperty("text") String text) + { + this.channel = requireNonNull(channel, "channel is null"); + this.text = requireNonNull(text, "text is null"); + } + + @JsonProperty + public String getChannel() + { + return channel; + } + + @JsonProperty + public String getText() + { + return text; + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackChatPostMessageResponse.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackChatPostMessageResponse.java new file mode 100644 index 000000000000..55b4a33e9ce9 --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackChatPostMessageResponse.java @@ -0,0 +1,41 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.twitter.presto.plugin.eventlistener.slack; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Optional; + +public class SlackChatPostMessageResponse + extends SlackResponse +{ + private final Optional message; + + @JsonCreator + public SlackChatPostMessageResponse( + @JsonProperty("ok") boolean ok, + @JsonProperty("message") Optional message, + @JsonProperty("error") Optional error) + { + super(ok, error); + this.message = message; + } + + @JsonProperty + public Optional getMessage() + { + return message; + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackImHistoryRequest.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackImHistoryRequest.java new file mode 100644 index 000000000000..7711df493587 --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackImHistoryRequest.java @@ -0,0 +1,46 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.twitter.presto.plugin.eventlistener.slack; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Optional; + +public class SlackImHistoryRequest +{ + private final String channel; + private final Optional latest; + + @JsonCreator + public SlackImHistoryRequest( + @JsonProperty("channel") String channel, + @JsonProperty("latest") Optional latest) + { + this.channel = channel; + this.latest = latest; + } + + @JsonProperty + public String getChannel() + { + return channel; + } + + @JsonProperty + public Optional getLatest() + { + return latest; + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackImHistoryResponse.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackImHistoryResponse.java new file mode 100644 index 000000000000..5a551e4e9cd6 --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackImHistoryResponse.java @@ -0,0 +1,60 @@ +/* + * 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.twitter.presto.plugin.eventlistener.slack; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; +import java.util.Optional; + +public class SlackImHistoryResponse + extends SlackResponse +{ + private final Optional latest; + private final Optional> messages; + private final Optional hasMore; + + @JsonCreator + public SlackImHistoryResponse( + @JsonProperty("ok") boolean ok, + @JsonProperty("latest") Optional latest, + @JsonProperty("messages") Optional> messages, + @JsonProperty("has_more") Optional hasMore, + @JsonProperty("error") Optional error) + { + super(ok, error); + this.latest = latest; + this.messages = messages; + this.hasMore = hasMore; + } + + @JsonProperty + public Optional getLatest() + { + return latest; + } + + @JsonProperty + public Optional> getMessages() + { + return messages; + } + + @JsonProperty("has_more") + public Optional getHasMore() + { + return hasMore; + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackImOpenRequest.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackImOpenRequest.java new file mode 100644 index 000000000000..c34f476d01d3 --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackImOpenRequest.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.twitter.presto.plugin.eventlistener.slack; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class SlackImOpenRequest +{ + private final String user; + + @JsonCreator + public SlackImOpenRequest( + @JsonProperty("user") String user) + { + this.user = user; + } + + @JsonProperty + public String getUser() + { + return user; + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackImOpenResponse.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackImOpenResponse.java new file mode 100644 index 000000000000..c46d3eade778 --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackImOpenResponse.java @@ -0,0 +1,41 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.twitter.presto.plugin.eventlistener.slack; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Optional; + +public class SlackImOpenResponse + extends SlackResponse +{ + private final Optional channel; + + @JsonCreator + public SlackImOpenResponse( + @JsonProperty("ok") boolean ok, + @JsonProperty("channel") Optional channel, + @JsonProperty("error") Optional error) + { + super(ok, error); + this.channel = channel; + } + + @JsonProperty + public Optional getChannel() + { + return channel; + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackMessage.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackMessage.java new file mode 100644 index 000000000000..0eb53e4effaf --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackMessage.java @@ -0,0 +1,44 @@ +/* + * 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.twitter.presto.plugin.eventlistener.slack; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class SlackMessage +{ + private final String text; + private final String ts; + + @JsonCreator + public SlackMessage( + @JsonProperty("text") String text, + @JsonProperty("ts") String ts) + { + this.text = text; + this.ts = ts; + } + + @JsonProperty + public String getText() + { + return text; + } + + @JsonProperty + public String getTs() + { + return ts; + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplate.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplate.java new file mode 100644 index 000000000000..c9df6e87bbf9 --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplate.java @@ -0,0 +1,51 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.twitter.presto.plugin.eventlistener.slack; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Optional; +import java.util.regex.Pattern; + +public class SlackNotificationTemplate +{ + private final String text; + private final Optional userRegex; + private final Optional eventRegex; + private final Optional stateRegex; + + @JsonCreator + public SlackNotificationTemplate( + @JsonProperty("text") String text, + @JsonProperty("user") Optional userRegex, + @JsonProperty("event") Optional eventRegex, + @JsonProperty("state") Optional stateRegex) + { + this.text = text; + this.userRegex = userRegex; + this.eventRegex = eventRegex; + this.stateRegex = stateRegex; + } + + public Optional match(String user, String event, String state) + { + if (userRegex.map(regex -> regex.matcher(user).matches()).orElse(true) && + eventRegex.map(regex -> regex.matcher(event).matches()).orElse(true) && + stateRegex.map(regex -> regex.matcher(state).matches()).orElse(true)) { + return Optional.of(text); + } + return Optional.empty(); + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplates.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplates.java new file mode 100644 index 000000000000..b538762727ab --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplates.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.twitter.presto.plugin.eventlistener.slack; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; +import java.util.Optional; + +public class SlackNotificationTemplates +{ + private List templates; + + @JsonCreator + public SlackNotificationTemplates( + @JsonProperty("templates") List templates) + { + this.templates = templates; + } + + @JsonProperty + public List getTemplates() + { + return templates; + } + + public Optional getText(String user, String event, String state) + { + return templates.stream() + .map(template -> template.match(user, event, state)) + .filter(Optional::isPresent) + .map(Optional::get) + .findFirst(); + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackResponse.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackResponse.java new file mode 100644 index 000000000000..275ab9d069fc --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackResponse.java @@ -0,0 +1,42 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.twitter.presto.plugin.eventlistener.slack; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Optional; + +public abstract class SlackResponse +{ + private final boolean ok; + private final Optional error; + + public SlackResponse(boolean ok, Optional error) + { + this.ok = ok; + this.error = error; + } + + @JsonProperty + public boolean isOk() + { + return ok; + } + + @JsonProperty + public Optional getError() + { + return error; + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackUser.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackUser.java new file mode 100644 index 000000000000..8bfc1b5f8619 --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackUser.java @@ -0,0 +1,44 @@ +/* + * 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.twitter.presto.plugin.eventlistener.slack; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class SlackUser +{ + private final String id; + private final String realName; + + @JsonCreator + public SlackUser( + @JsonProperty("id") String id, + @JsonProperty("real_name") String realName) + { + this.id = id; + this.realName = realName; + } + + @JsonProperty + public String getId() + { + return id; + } + + @JsonProperty("real_name") + public String getRealName() + { + return realName; + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackUsersLookupByEmailResponse.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackUsersLookupByEmailResponse.java new file mode 100644 index 000000000000..28c43b352223 --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackUsersLookupByEmailResponse.java @@ -0,0 +1,41 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.twitter.presto.plugin.eventlistener.slack; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Optional; + +public class SlackUsersLookupByEmailResponse + extends SlackResponse +{ + private final Optional user; + + @JsonCreator + public SlackUsersLookupByEmailResponse( + @JsonProperty("ok") boolean ok, + @JsonProperty("user") Optional user, + @JsonProperty("error") Optional error) + { + super(ok, error); + this.user = user; + } + + @JsonProperty + public Optional getUser() + { + return user; + } +} diff --git a/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/TestTwitterEventListenerConfig.java b/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/TestTwitterEventListenerConfig.java new file mode 100644 index 000000000000..9f33354fa08c --- /dev/null +++ b/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/TestTwitterEventListenerConfig.java @@ -0,0 +1,66 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.twitter.presto.plugin.eventlistener; + +import com.google.common.collect.ImmutableMap; +import com.google.common.net.HostAndPort; +import io.airlift.configuration.testing.ConfigAssertions; +import org.testng.annotations.Test; + +import java.net.URI; +import java.util.Map; + +public class TestTwitterEventListenerConfig +{ + @Test + public void testDefaults() + { + ConfigAssertions.assertRecordedDefaults(ConfigAssertions.recordDefaults(TwitterEventListenerConfig.class) + .setScribeCategory(null) + .setSlackConfigFile(null) + .setSlackEmailTemplate(null) + .setSlackHttpProxy(null) + .setSlackNotificationTemplateFile(null) + .setKnowledgeBaseFile(null) + .setSlackUri(null) + .setSlackUsers(null)); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = new ImmutableMap.Builder() + .put("event-listener.scribe-category", "test") + .put("event-listener.knowledge-base-file", "/etc/config/knowledge.json") + .put("event-listener.slack-config-file", "/etc/config/slack.json") + .put("event-listener.slack-email-template", "${USER}@domain.top") + .put("event-listener.slack-http-proxy", "localhost:1008") + .put("event-listener.slack-notification-template-file", "/etc/config/notification.json") + .put("event-listener.slack-uri", "https://slack.com") + .put("event-listener.slack-users", "user1|user2") + .build(); + + TwitterEventListenerConfig expected = new TwitterEventListenerConfig() + .setScribeCategory("test") + .setKnowledgeBaseFile("/etc/config/knowledge.json") + .setSlackConfigFile("/etc/config/slack.json") + .setSlackEmailTemplate("${USER}@domain.top") + .setSlackHttpProxy(HostAndPort.fromString("localhost:1008")) + .setSlackNotificationTemplateFile("/etc/config/notification.json") + .setSlackUri(URI.create("https://slack.com")) + .setSlackUsers("user1|user2"); + + ConfigAssertions.assertFullMapping(properties, expected); + } +} diff --git a/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/TestTwitterEventListenerPlugin.java b/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/TestTwitterEventListenerPlugin.java new file mode 100644 index 000000000000..33938d407bb3 --- /dev/null +++ b/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/TestTwitterEventListenerPlugin.java @@ -0,0 +1,56 @@ +/* + * 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.twitter.presto.plugin.eventlistener; + +import com.facebook.presto.spi.Plugin; +import com.facebook.presto.spi.eventlistener.EventListener; +import com.facebook.presto.spi.eventlistener.EventListenerFactory; +import com.google.common.collect.ImmutableMap; +import org.testng.annotations.Test; + +import java.util.Map; +import java.util.ServiceLoader; + +import static com.google.common.collect.Iterables.getOnlyElement; +import static io.airlift.testing.Assertions.assertInstanceOf; +import static org.testng.Assert.assertNotNull; + +public class TestTwitterEventListenerPlugin +{ + @Test + public void testPlugin() + { + TwitterEventListenerPlugin plugin = loadPlugin(TwitterEventListenerPlugin.class); + + EventListenerFactory factory = getOnlyElement(plugin.getEventListenerFactories()); + assertInstanceOf(factory, TwitterEventListenerFactory.class); + + Map config = ImmutableMap.of(); + + EventListener eventListener = factory.create(config); + assertNotNull(eventListener); + assertInstanceOf(eventListener, TwitterEventListener.class); + } + + @SuppressWarnings("unchecked") + private static T loadPlugin(Class clazz) + { + for (Plugin plugin : ServiceLoader.load(Plugin.class)) { + if (clazz.isInstance(plugin)) { + return (T) plugin; + } + } + throw new AssertionError("did not find plugin: " + clazz.getName()); + } +} From 7f856a0d5b8f53594634ccbe5d60486cc82dc195 Mon Sep 17 00:00:00 2001 From: Hao Luo Date: Thu, 15 Nov 2018 11:47:12 -0800 Subject: [PATCH 310/331] Add additional travis job for product test to avoid timeout (#179) * split travis job PRODUCT_TESTS_SPECIFIC_ENVIRONMENT into 2 smaller jobs to avoid timeout --- .travis.yml | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/.travis.yml b/.travis.yml index 1054d8d85a50..a047d0b2edd6 100644 --- a/.travis.yml +++ b/.travis.yml @@ -21,6 +21,7 @@ env: - PRODUCT_TESTS_BASIC_ENVIRONMENT=true - PRODUCT_TESTS_SPECIFIC_ENVIRONMENT=true - PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_2=true + - PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_3=true - HIVE_TESTS=true - KUDU_TESTS=true @@ -61,7 +62,7 @@ install: ./mvnw install $MAVEN_FAST_INSTALL -P !twitter-modules -pl '!presto-docs,!presto-server,!presto-server-rpm' fi - | - if [[ -v PRODUCT_TESTS_BASIC_ENVIRONMENT || -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT || -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_2 ]]; then + if [[ -v PRODUCT_TESTS_BASIC_ENVIRONMENT || -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT || -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_2 || -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_3 ]]; then ./mvnw install $MAVEN_FAST_INSTALL -P !twitter-modules -pl '!presto-docs,!presto-server-rpm' fi - | @@ -123,12 +124,12 @@ script: singlenode-hdfs-impersonation -g storage_formats,cli,hdfs_impersonation fi - | - if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then + if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_3 ]]; then presto-product-tests/bin/run_on_docker.sh \ singlenode-kerberos-hdfs-impersonation -g storage_formats,cli,hdfs_impersonation,authorization,hive_file_header fi - | - if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then + if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_3 ]]; then presto-product-tests/bin/run_on_docker.sh \ singlenode-kerberos-hdfs-impersonation-cross-realm -g storage_formats,cli,hdfs_impersonation fi @@ -165,7 +166,7 @@ script: presto-product-tests/bin/run_on_docker.sh \ singlenode-cassandra -g cassandra fi - if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT ]]; then + if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_3 ]]; then presto-product-tests/bin/run_on_docker.sh \ multinode-tls-kerberos -g cli,group-by,join,tls fi From aef60d5bed19a57c2c969f45dc24be2c1643cc94 Mon Sep 17 00:00:00 2001 From: Hao Luo Date: Fri, 16 Nov 2018 12:38:01 -0800 Subject: [PATCH 311/331] Add maintenance coordinator module for Aurora SlaPolicy Implement the SLA requirement described in the following wiki page: http://aurora.apache.org/documentation/latest/features/sla-requirements/#coordinator-based --- .../facebook/presto/server/ServerConfig.java | 13 ++ .../presto/server/ServerMainModule.java | 4 + .../presto/maintenance/ForMaintenance.java | 31 ++++ .../MaintenanceCoordinatorModule.java | 31 ++++ .../MaintenanceCoordinatorResource.java | 159 ++++++++++++++++ .../presto/server/TestServerConfig.java | 3 + presto-tests/pom.xml | 5 + .../presto/tests/TestMaintenanceModule.java | 172 ++++++++++++++++++ 8 files changed, 418 insertions(+) create mode 100644 presto-main/src/main/java/com/twitter/presto/maintenance/ForMaintenance.java create mode 100644 presto-main/src/main/java/com/twitter/presto/maintenance/MaintenanceCoordinatorModule.java create mode 100644 presto-main/src/main/java/com/twitter/presto/maintenance/MaintenanceCoordinatorResource.java create mode 100644 presto-tests/src/test/java/com/twitter/presto/tests/TestMaintenanceModule.java diff --git a/presto-main/src/main/java/com/facebook/presto/server/ServerConfig.java b/presto-main/src/main/java/com/facebook/presto/server/ServerConfig.java index bbe127c6979c..19a25ce502ab 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/ServerConfig.java +++ b/presto-main/src/main/java/com/facebook/presto/server/ServerConfig.java @@ -28,6 +28,7 @@ public class ServerConfig private boolean includeExceptionInResponse = true; private Duration gracePeriod = new Duration(2, MINUTES); private boolean enhancedErrorReporting = true; + private boolean maintenanceCoordinator; public boolean isCoordinator() { @@ -106,4 +107,16 @@ public ServerConfig setEnhancedErrorReporting(boolean value) this.enhancedErrorReporting = value; return this; } + + public boolean isMaintenanceCoordinator() + { + return maintenanceCoordinator; + } + + @Config("maintenance.coordinator") + public ServerConfig setMaintenanceCoordinator(boolean maintenanceCoordinator) + { + this.maintenanceCoordinator = maintenanceCoordinator; + return this; + } } diff --git a/presto-main/src/main/java/com/facebook/presto/server/ServerMainModule.java b/presto-main/src/main/java/com/facebook/presto/server/ServerMainModule.java index bcbffc7ba53e..ec0ee8fb6dad 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/ServerMainModule.java +++ b/presto-main/src/main/java/com/facebook/presto/server/ServerMainModule.java @@ -137,6 +137,7 @@ import com.google.inject.Provides; import com.google.inject.Scopes; import com.google.inject.TypeLiteral; +import com.twitter.presto.maintenance.MaintenanceCoordinatorModule; import io.airlift.concurrent.BoundedExecutor; import io.airlift.configuration.AbstractConfigurationAwareModule; import io.airlift.slice.Slice; @@ -193,6 +194,9 @@ protected void setup(Binder binder) if (serverConfig.isCoordinator()) { install(new CoordinatorModule()); + if (serverConfig.isMaintenanceCoordinator()) { + install(new MaintenanceCoordinatorModule()); + } } else { // Install no-op session supplier on workers, since only coordinators create sessions. diff --git a/presto-main/src/main/java/com/twitter/presto/maintenance/ForMaintenance.java b/presto-main/src/main/java/com/twitter/presto/maintenance/ForMaintenance.java new file mode 100644 index 000000000000..66aff04827f6 --- /dev/null +++ b/presto-main/src/main/java/com/twitter/presto/maintenance/ForMaintenance.java @@ -0,0 +1,31 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.twitter.presto.maintenance; + +import javax.inject.Qualifier; + +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +import static java.lang.annotation.ElementType.FIELD; +import static java.lang.annotation.ElementType.METHOD; +import static java.lang.annotation.ElementType.PARAMETER; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +@Retention(RUNTIME) +@Target({FIELD, PARAMETER, METHOD}) +@Qualifier +public @interface ForMaintenance +{ +} diff --git a/presto-main/src/main/java/com/twitter/presto/maintenance/MaintenanceCoordinatorModule.java b/presto-main/src/main/java/com/twitter/presto/maintenance/MaintenanceCoordinatorModule.java new file mode 100644 index 000000000000..70a9750d551b --- /dev/null +++ b/presto-main/src/main/java/com/twitter/presto/maintenance/MaintenanceCoordinatorModule.java @@ -0,0 +1,31 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.twitter.presto.maintenance; + +import com.google.inject.Binder; +import com.google.inject.Module; + +import static io.airlift.http.client.HttpClientBinder.httpClientBinder; +import static io.airlift.jaxrs.JaxrsBinder.jaxrsBinder; + +public class MaintenanceCoordinatorModule + implements Module +{ + @Override + public void configure(Binder binder) + { + httpClientBinder(binder).bindHttpClient("maintenance", ForMaintenance.class); + jaxrsBinder(binder).bind(MaintenanceCoordinatorResource.class); + } +} diff --git a/presto-main/src/main/java/com/twitter/presto/maintenance/MaintenanceCoordinatorResource.java b/presto-main/src/main/java/com/twitter/presto/maintenance/MaintenanceCoordinatorResource.java new file mode 100644 index 000000000000..1624967eb432 --- /dev/null +++ b/presto-main/src/main/java/com/twitter/presto/maintenance/MaintenanceCoordinatorResource.java @@ -0,0 +1,159 @@ +/* + * 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.twitter.presto.maintenance; + +import com.facebook.presto.spi.NodeState; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import io.airlift.http.client.HttpClient; +import io.airlift.http.client.Request; +import io.airlift.json.JsonCodec; +import io.airlift.log.Logger; + +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.WebApplicationException; +import javax.ws.rs.core.Response; + +import java.io.IOException; +import java.net.URI; + +import static com.google.common.net.MediaType.JSON_UTF_8; +import static io.airlift.http.client.HttpUriBuilder.uriBuilderFrom; +import static io.airlift.http.client.JsonBodyGenerator.jsonBodyGenerator; +import static io.airlift.http.client.JsonResponseHandler.createJsonResponseHandler; +import static io.airlift.http.client.Request.Builder.prepareGet; +import static io.airlift.http.client.Request.Builder.preparePut; +import static io.airlift.http.client.StatusResponseHandler.createStatusResponseHandler; +import static io.airlift.json.JsonCodec.jsonCodec; +import static java.util.Objects.requireNonNull; +import static javax.ws.rs.core.HttpHeaders.CONTENT_TYPE; +import static javax.ws.rs.core.MediaType.TEXT_PLAIN_TYPE; + +@Path("/canDrain") +public class MaintenanceCoordinatorResource +{ + private static final Logger log = Logger.get(MaintenanceCoordinatorResource.class); + private static final JsonCodec NODE_STATE_CODEC = jsonCodec(NodeState.class); + private static final ObjectMapper jsonObjectMapper = new ObjectMapper(); + private final HttpClient httpClient; + + @Inject + public MaintenanceCoordinatorResource(@ForMaintenance HttpClient httpClient) + { + this.httpClient = requireNonNull(httpClient, "httpClient is null"); + } + + @POST + public DrainResponse canDrain(String jsonString) + { + URI nodeUri = extractHostUri(jsonString); + log.info("Try draining node : " + nodeUri); + + // check the state of the target node + NodeState state = getNodeState(nodeUri); + + // if the node is active, we send the shutdown request + if (state == NodeState.ACTIVE) { + shutdownNode(nodeUri); + } + return new DrainResponse(false); + + // We should NEVER return "true" to drain request. What will happen is that the first request will request graceful shutdown in the target and the target node + // state will transfer from ACTIVE to SHUTTING_DOWN. When the shutdown is completed, getNodeState() will fail and the exception will propagate to aurora COp. + // COp always list active tasks before requesting drain, but there is a race condition which may expose a small window where the task finishes between COp list the + // active tasks and maintenance coordinator query the state of the target. COp will treat the exception as a NO, and the next retry should proceed without requesting + // maintenance coordinator. + } + + private NodeState getNodeState(URI nodeUri) + { + // synchronously send SHUTTING_DOWN request to worker node + Request request = prepareGet() + .setUri(getNodeStateUri(nodeUri)) + .setHeader(CONTENT_TYPE, JSON_UTF_8.toString()) + .build(); + + NodeState nodeState = httpClient.execute(request, createJsonResponseHandler(NODE_STATE_CODEC)); + + log.info("Node " + nodeUri + " in state : " + nodeState); + return nodeState; + } + + private void shutdownNode(URI nodeUri) + { + log.info("Shutting down node : " + nodeUri); + Request request = preparePut() + .setUri(getNodeStateUri(nodeUri)) + .setHeader(CONTENT_TYPE, JSON_UTF_8.toString()) + .setBodyGenerator(jsonBodyGenerator(jsonCodec(NodeState.class), NodeState.SHUTTING_DOWN)) + .build(); + + httpClient.execute(request, createStatusResponseHandler()); + } + + // extract the worker node URI from the request body + private URI extractHostUri(String message) + { + try { + JsonNode jsonRoot = jsonObjectMapper.readTree(message); + String hostName = jsonRoot + .get("taskConfig") + .get("assignedTask") + .get("slaveHost") + .asText(); + int port = jsonRoot + .get("taskConfig") + .get("assignedTask") + .get("assignedPorts") + .get("http") + .asInt(); + return URI.create("http://" + hostName + ":" + port); + } + catch (IOException e) { + String errorMessage = "Malformed Json body in drain request " + message; + log.warn(e, errorMessage); + throw new WebApplicationException( + Response.status(Response.Status.BAD_REQUEST) + .type(TEXT_PLAIN_TYPE) + .entity(errorMessage) + .build()); + } + } + + private URI getNodeStateUri(URI nodeUri) + { + return uriBuilderFrom(nodeUri).appendPath("/v1/info/state").build(); + } + + public static class DrainResponse + { + private final boolean drain; + + @JsonCreator + public DrainResponse(@JsonProperty("drain") boolean drain) + { + this.drain = drain; + } + + @JsonProperty + public boolean getDrain() + { + return drain; + } + } +} diff --git a/presto-main/src/test/java/com/facebook/presto/server/TestServerConfig.java b/presto-main/src/test/java/com/facebook/presto/server/TestServerConfig.java index de9c5c7fbd78..a435def1e527 100644 --- a/presto-main/src/test/java/com/facebook/presto/server/TestServerConfig.java +++ b/presto-main/src/test/java/com/facebook/presto/server/TestServerConfig.java @@ -35,6 +35,7 @@ public void testDefaults() .setDataSources(null) .setIncludeExceptionInResponse(true) .setGracePeriod(new Duration(2, MINUTES)) + .setMaintenanceCoordinator(false) .setEnhancedErrorReporting(true)); } @@ -48,6 +49,7 @@ public void testExplicitPropertyMappings() .put("http.include-exception-in-response", "false") .put("shutdown.grace-period", "5m") .put("sql.parser.enhanced-error-reporting", "false") + .put("maintenance.coordinator", "true") .build(); ServerConfig expected = new ServerConfig() @@ -56,6 +58,7 @@ public void testExplicitPropertyMappings() .setDataSources("jmx") .setIncludeExceptionInResponse(false) .setGracePeriod(new Duration(5, MINUTES)) + .setMaintenanceCoordinator(true) .setEnhancedErrorReporting(false); assertFullMapping(properties, expected); diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 552656cc1570..c91a6155dec8 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -181,6 +181,11 @@ runtime + + io.airlift + http-client + + com.facebook.presto diff --git a/presto-tests/src/test/java/com/twitter/presto/tests/TestMaintenanceModule.java b/presto-tests/src/test/java/com/twitter/presto/tests/TestMaintenanceModule.java new file mode 100644 index 000000000000..d7af324f237f --- /dev/null +++ b/presto-tests/src/test/java/com/twitter/presto/tests/TestMaintenanceModule.java @@ -0,0 +1,172 @@ +/* + * 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.twitter.presto.tests; + +import com.facebook.presto.Session; +import com.facebook.presto.execution.TaskManager; +import com.facebook.presto.server.testing.TestingPrestoServer; +import com.facebook.presto.spi.NodeState; +import com.facebook.presto.tests.DistributedQueryRunner; +import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import com.twitter.presto.maintenance.MaintenanceCoordinatorResource.DrainResponse; +import io.airlift.http.client.FullJsonResponseHandler.JsonResponse; +import io.airlift.http.client.HttpClient; +import io.airlift.http.client.Request; +import io.airlift.http.client.jetty.JettyHttpClient; +import io.airlift.json.JsonCodec; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.net.URI; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static com.facebook.presto.execution.QueryState.FINISHED; +import static com.facebook.presto.memory.TestMemoryManager.createQueryRunner; +import static com.facebook.presto.testing.TestingSession.testSessionBuilder; +import static com.google.common.net.MediaType.JSON_UTF_8; +import static io.airlift.http.client.FullJsonResponseHandler.createFullJsonResponseHandler; +import static io.airlift.http.client.HttpStatus.INTERNAL_SERVER_ERROR; +import static io.airlift.http.client.HttpStatus.OK; +import static io.airlift.http.client.HttpUriBuilder.uriBuilderFrom; +import static io.airlift.http.client.JsonResponseHandler.createJsonResponseHandler; +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.json.JsonCodec.jsonCodec; +import static java.lang.Thread.sleep; +import static java.nio.charset.StandardCharsets.UTF_8; +import static java.util.concurrent.Executors.newCachedThreadPool; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static javax.ws.rs.core.HttpHeaders.CONTENT_TYPE; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; + +public class TestMaintenanceModule +{ + private static final long SHUTDOWN_TIMEOUT_MILLIS = 240_000; + private static final JsonCodec DRAIN_RESPONSE_CODEC = JsonCodec.jsonCodec(DrainResponse.class); + private static final JsonCodec NODE_STATE_CODEC = jsonCodec(NodeState.class); + + private static final Session TINY_SESSION = testSessionBuilder() + .setCatalog("tpch") + .setSchema("tiny") + .build(); + private ListeningExecutorService executor; + private HttpClient client; + + @BeforeClass + public void setUp() + { + executor = MoreExecutors.listeningDecorator(newCachedThreadPool()); + client = new JettyHttpClient(); + } + + @AfterClass(alwaysRun = true) + public void shutdown() + { + executor.shutdownNow(); + } + + @Test(timeOut = SHUTDOWN_TIMEOUT_MILLIS) + public void testMaintenanceModule() + throws Exception + { + Map properties = ImmutableMap.builder() + .put("node-scheduler.include-coordinator", "false") + .put("shutdown.grace-period", "1s") + .put("maintenance.coordinator", "true") + .build(); + + try (DistributedQueryRunner queryRunner = createQueryRunner(TINY_SESSION, properties)) { + List> queryFutures = new ArrayList<>(); + for (int i = 0; i < 5; i++) { + queryFutures.add(executor.submit(() -> queryRunner.execute("SELECT COUNT(*), clerk FROM orders GROUP BY clerk"))); + } + + // pick a random worker for maintenance + TestingPrestoServer worker = queryRunner.getServers() + .stream() + .filter(server -> !server.isCoordinator()) + .findAny() + .get(); + + TaskManager taskManager = worker.getTaskManager(); + // wait until tasks show up on the worker + while (taskManager.getAllTaskInfo().isEmpty()) { + MILLISECONDS.sleep(500); + } + + // try drain the worker + while (true) { + JsonResponse response = tryDrain(queryRunner.getCoordinator().getBaseUrl(), worker.getBaseUrl()); + if (response.getStatusCode() == OK.code()) { + assertTrue(response.getValue().getDrain() == false); + // check the remote node state to make sure node is shutting down + assertTrue(getNodeState(worker.getBaseUrl()) == NodeState.SHUTTING_DOWN); + } + else if (response.getStatusCode() == INTERNAL_SERVER_ERROR.code()) { + // 500 code indicates that the node is down and unreachable + break; + } + sleep(1000); + } + + // HACK: we can't simulate lifecycle of individu + Futures.allAsList(queryFutures).get(); + + queryRunner.getCoordinator().getQueryManager().getAllQueryInfo().stream() + .forEach(x -> assertEquals(x.getState(), FINISHED)); + + TestingPrestoServer.TestShutdownAction shutdownAction = (TestingPrestoServer.TestShutdownAction) worker.getShutdownAction(); + shutdownAction.waitForShutdownComplete(SHUTDOWN_TIMEOUT_MILLIS); + assertTrue(shutdownAction.isWorkerShutdown()); + } + } + + private JsonResponse tryDrain(URI coordinatorUri, URI targetUri) + { + StringBuilder stringBuilder = new StringBuilder(); + stringBuilder.append("{ \"taskConfig\": { \"assignedTask\": { \"slaveHost\": \"") + .append(targetUri.getHost()) + .append("\", \"assignedPorts\": { \"http\": ") + .append(targetUri.getPort()) + .append(" } } } }"); + + Request request = preparePost() + .setUri(uriBuilderFrom(coordinatorUri).appendPath("/canDrain").build()) + .setHeader(CONTENT_TYPE, JSON_UTF_8.toString()) + .setBodyGenerator(createStaticBodyGenerator(stringBuilder.toString(), UTF_8)) + .build(); + return client.execute(request, createFullJsonResponseHandler(DRAIN_RESPONSE_CODEC)); + } + + private NodeState getNodeState(URI nodeUri) + { + URI nodeStateUri = uriBuilderFrom(nodeUri).appendPath("/v1/info/state").build(); + // synchronously send SHUTTING_DOWN request to worker node + Request request = prepareGet() + .setUri(nodeStateUri) + .setHeader(CONTENT_TYPE, JSON_UTF_8.toString()) + .build(); + + return client.execute(request, createJsonResponseHandler(NODE_STATE_CODEC)); + } +} From 621c8f84c409368bd99a7a249cbb5b5694af233d Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 16 Nov 2018 20:30:14 -0800 Subject: [PATCH 312/331] Add principal match option for notification template (#180) --- .../twitter/presto/plugin/eventlistener/slack/SlackBot.java | 2 +- .../eventlistener/slack/SlackNotificationTemplate.java | 6 +++++- .../eventlistener/slack/SlackNotificationTemplates.java | 4 ++-- 3 files changed, 8 insertions(+), 4 deletions(-) diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBot.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBot.java index 81359f69fb0e..93a9ecf95474 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBot.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBot.java @@ -139,7 +139,7 @@ private void handleSlackNotification(String event, String user, String queryId, if (!slackUsers.matcher(user).matches()) { return; } - Optional template = notificationTemplates.getText(user, event, state); + Optional template = notificationTemplates.getText(user, principal.orElse(DASH), event, state); if (!template.isPresent()) { return; } diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplate.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplate.java index c9df6e87bbf9..f375f61f617a 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplate.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplate.java @@ -23,6 +23,7 @@ public class SlackNotificationTemplate { private final String text; private final Optional userRegex; + private final Optional principalRegex; private final Optional eventRegex; private final Optional stateRegex; @@ -30,18 +31,21 @@ public class SlackNotificationTemplate public SlackNotificationTemplate( @JsonProperty("text") String text, @JsonProperty("user") Optional userRegex, + @JsonProperty("principal") Optional principalRegex, @JsonProperty("event") Optional eventRegex, @JsonProperty("state") Optional stateRegex) { this.text = text; this.userRegex = userRegex; + this.principalRegex = principalRegex; this.eventRegex = eventRegex; this.stateRegex = stateRegex; } - public Optional match(String user, String event, String state) + public Optional match(String user, String principal, String event, String state) { if (userRegex.map(regex -> regex.matcher(user).matches()).orElse(true) && + principalRegex.map(regex -> regex.matcher(principal).matches()).orElse(true) && eventRegex.map(regex -> regex.matcher(event).matches()).orElse(true) && stateRegex.map(regex -> regex.matcher(state).matches()).orElse(true)) { return Optional.of(text); diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplates.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplates.java index b538762727ab..fce4caaaf4be 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplates.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplates.java @@ -36,10 +36,10 @@ public List getTemplates() return templates; } - public Optional getText(String user, String event, String state) + public Optional getText(String user, String principal, String event, String state) { return templates.stream() - .map(template -> template.match(user, event, state)) + .map(template -> template.match(user, principal, event, state)) .filter(Optional::isPresent) .map(Optional::get) .findFirst(); From 4babc43954cb01f94a353b4ad75cdd8b2cf2e578 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Fri, 16 Nov 2018 20:30:48 -0800 Subject: [PATCH 313/331] Add test for TwitterEventListener (#181) --- twitter-eventlistener-plugin/pom.xml | 18 ++ .../TestTwitterEventListener.java | 175 ++++++++++++++++++ .../TestingTwitterEventListenerPlugin.java | 64 +++++++ 3 files changed, 257 insertions(+) create mode 100644 twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/TestTwitterEventListener.java create mode 100644 twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/TestingTwitterEventListenerPlugin.java diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index c0892e67e5e7..165123754597 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -177,5 +177,23 @@ testing test + + + com.facebook.presto + presto-tests + test + + + + com.facebook.presto + presto-main + test + + + + com.facebook.presto + presto-tpch + test + diff --git a/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/TestTwitterEventListener.java b/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/TestTwitterEventListener.java new file mode 100644 index 000000000000..02b8e019e851 --- /dev/null +++ b/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/TestTwitterEventListener.java @@ -0,0 +1,175 @@ +/* + * 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.twitter.presto.plugin.eventlistener; + +import com.facebook.presto.Session; +import com.facebook.presto.spi.eventlistener.QueryCompletedEvent; +import com.facebook.presto.spi.eventlistener.QueryCreatedEvent; +import com.facebook.presto.spi.eventlistener.SplitCompletedEvent; +import com.facebook.presto.testing.MaterializedResult; +import com.facebook.presto.tests.DistributedQueryRunner; +import com.facebook.presto.tpch.TpchPlugin; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static com.facebook.presto.testing.TestingSession.testSessionBuilder; +import static com.google.common.collect.Iterables.getOnlyElement; +import static org.testng.Assert.assertEquals; + +@Test(singleThreaded = true) +public class TestTwitterEventListener +{ + private static final int SPLITS_PER_NODE = 3; + private final TestingTwitterEventHandler handler = new TestingTwitterEventHandler(); + + private DistributedQueryRunner queryRunner; + private Session session; + + @BeforeClass + private void setUp() + throws Exception + { + session = testSessionBuilder() + .setSystemProperty("task_concurrency", "1") + .setCatalog("tpch") + .setSchema("tiny") + .setClientInfo("{\"clientVersion\":\"testVersion\"}") + .build(); + queryRunner = new DistributedQueryRunner(session, 1); + queryRunner.installPlugin(new TpchPlugin()); + queryRunner.installPlugin(new TestingTwitterEventListenerPlugin(handler)); + queryRunner.createCatalog("tpch", "tpch", ImmutableMap.of("tpch.splits-per-node", Integer.toString(SPLITS_PER_NODE))); + } + + @AfterClass(alwaysRun = true) + private void tearDown() + { + queryRunner.close(); + queryRunner = null; + } + + @Test + public void testConstantQuery() + throws Exception + { + // QueryCreated: 1, QueryCompleted: 1, Splits: 1 + runQueryAndWaitForEvents("SELECT 1", 3); + + QueryCreatedEvent queryCreatedEvent = getOnlyElement(handler.getQueryCreatedEvents()); + assertEquals(queryCreatedEvent.getContext().getServerVersion(), "testversion"); + assertEquals(queryCreatedEvent.getContext().getServerAddress(), "127.0.0.1"); + assertEquals(queryCreatedEvent.getContext().getEnvironment(), "testing"); + assertEquals(queryCreatedEvent.getContext().getClientInfo().get(), "{\"clientVersion\":\"testVersion\"}"); + assertEquals(queryCreatedEvent.getMetadata().getQuery(), "SELECT 1"); + + QueryCompletedEvent queryCompletedEvent = getOnlyElement(handler.getQueryCompletedEvents()); + assertEquals(queryCompletedEvent.getStatistics().getTotalRows(), 0L); + assertEquals(queryCompletedEvent.getContext().getClientInfo().get(), "{\"clientVersion\":\"testVersion\"}"); + assertEquals(queryCreatedEvent.getMetadata().getQueryId(), queryCompletedEvent.getMetadata().getQueryId()); + + List splitCompletedEvents = handler.getSplitCompletedEvents(); + assertEquals(splitCompletedEvents.get(0).getQueryId(), queryCompletedEvent.getMetadata().getQueryId()); + assertEquals(splitCompletedEvents.get(0).getStatistics().getCompletedPositions(), 1); + } + + private MaterializedResult runQueryAndWaitForEvents(String sql, int numEventsExpected) + throws Exception + { + handler.initialize(numEventsExpected); + MaterializedResult result = queryRunner.execute(session, sql); + handler.waitForEvents(10); + + return result; + } + + static class TestingTwitterEventHandler + implements TwitterEventHandler + { + private ImmutableList.Builder queryCreatedEvents; + private ImmutableList.Builder queryCompletedEvents; + private ImmutableList.Builder splitCompletedEvents; + + private CountDownLatch eventsLatch; + + public synchronized void initialize(int numEvents) + { + queryCreatedEvents = ImmutableList.builder(); + queryCompletedEvents = ImmutableList.builder(); + splitCompletedEvents = ImmutableList.builder(); + + eventsLatch = new CountDownLatch(numEvents); + } + + public void waitForEvents(int timeoutSeconds) + throws InterruptedException + { + eventsLatch.await(timeoutSeconds, TimeUnit.SECONDS); + } + + public synchronized void addQueryCreated(QueryCreatedEvent event) + { + queryCreatedEvents.add(event); + eventsLatch.countDown(); + } + + public synchronized void addQueryCompleted(QueryCompletedEvent event) + { + queryCompletedEvents.add(event); + eventsLatch.countDown(); + } + + public synchronized void addSplitCompleted(SplitCompletedEvent event) + { + splitCompletedEvents.add(event); + eventsLatch.countDown(); + } + + public List getQueryCreatedEvents() + { + return queryCreatedEvents.build(); + } + + public List getQueryCompletedEvents() + { + return queryCompletedEvents.build(); + } + + public List getSplitCompletedEvents() + { + return splitCompletedEvents.build(); + } + + public void handleQueryCreated(QueryCreatedEvent queryCreatedEvent) + { + addQueryCreated(queryCreatedEvent); + } + + public void handleQueryCompleted(QueryCompletedEvent queryCompletedEvent) + { + addQueryCompleted(queryCompletedEvent); + } + + public void handleSplitCompleted(SplitCompletedEvent splitCompletedEvent) + { + addSplitCompleted(splitCompletedEvent); + } + } +} diff --git a/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/TestingTwitterEventListenerPlugin.java b/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/TestingTwitterEventListenerPlugin.java new file mode 100644 index 000000000000..c181dd601421 --- /dev/null +++ b/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/TestingTwitterEventListenerPlugin.java @@ -0,0 +1,64 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.twitter.presto.plugin.eventlistener; + +import com.facebook.presto.spi.Plugin; +import com.facebook.presto.spi.eventlistener.EventListener; +import com.facebook.presto.spi.eventlistener.EventListenerFactory; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; + +import java.util.Map; + +import static java.util.Objects.requireNonNull; + +public class TestingTwitterEventListenerPlugin + implements Plugin +{ + private EventListenerFactory factory; + + public TestingTwitterEventListenerPlugin(TwitterEventHandler handler) + { + this.factory = new TestingTwitterEventListenerFactory(requireNonNull(handler, "handler is null")); + } + + @Override + public Iterable getEventListenerFactories() + { + return ImmutableList.of(factory); + } + + private class TestingTwitterEventListenerFactory + implements EventListenerFactory + { + private TwitterEventHandler handler; + + public TestingTwitterEventListenerFactory(TwitterEventHandler handler) + { + this.handler = requireNonNull(handler, "handler is null"); + } + + @Override + public String getName() + { + return "testing-twitter-event-listener"; + } + + @Override + public EventListener create(Map config) + { + return new TwitterEventListener(ImmutableSet.of(handler)); + } + } +} From 968418a6e2b690bc06ee8b8e978812f36aeda622 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 20 Nov 2018 13:35:20 -0800 Subject: [PATCH 314/331] Add tests for QueryCompletedEventScriber and SlackBot (#182) * Add tests for QueryCompletedEventScriber * Add test for SlackBot --- .travis.yml | 16 +- pom.xml | 12 +- twitter-eventlistener-plugin/pom.xml | 103 +++---- .../scriber/QueryCompletedEventScriber.java | 5 + .../eventlistener/scriber/TwitterScriber.java | 2 +- .../plugin/eventlistener/slack/SlackBot.java | 16 +- .../src/main/thrift/presto.thrift | 112 +++++++ .../TestingTwitterEventListenerPlugin.java | 16 +- .../TestQueryCompletedEventScriber.java | 209 +++++++++++++ .../eventlistener/slack/TestSlackBot.java | 274 ++++++++++++++++++ .../slack/TestingSlackResource.java | 163 +++++++++++ .../src/test/resources/slackCredentials.json | 3 + .../test/resources/slackNotifications.json | 18 ++ 13 files changed, 870 insertions(+), 79 deletions(-) create mode 100644 twitter-eventlistener-plugin/src/main/thrift/presto.thrift create mode 100644 twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/scriber/TestQueryCompletedEventScriber.java create mode 100644 twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/slack/TestSlackBot.java create mode 100644 twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/slack/TestingSlackResource.java create mode 100644 twitter-eventlistener-plugin/src/test/resources/slackCredentials.json create mode 100644 twitter-eventlistener-plugin/src/test/resources/slackNotifications.json diff --git a/.travis.yml b/.travis.yml index a047d0b2edd6..18a803fcb62b 100644 --- a/.travis.yml +++ b/.travis.yml @@ -55,19 +55,19 @@ install: - ./mvnw -v - | if [[ -v TEST_SPECIFIC_MODULES ]]; then - ./mvnw install $MAVEN_FAST_INSTALL -P !twitter-modules -pl $TEST_SPECIFIC_MODULES -am + ./mvnw install $MAVEN_FAST_INSTALL -pl $TEST_SPECIFIC_MODULES -am fi - | if [[ -v TEST_OTHER_MODULES ]]; then - ./mvnw install $MAVEN_FAST_INSTALL -P !twitter-modules -pl '!presto-docs,!presto-server,!presto-server-rpm' + ./mvnw install $MAVEN_FAST_INSTALL -pl '!presto-docs,!presto-server,!presto-server-rpm' fi - | if [[ -v PRODUCT_TESTS_BASIC_ENVIRONMENT || -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT || -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_2 || -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_3 ]]; then - ./mvnw install $MAVEN_FAST_INSTALL -P !twitter-modules -pl '!presto-docs,!presto-server-rpm' + ./mvnw install $MAVEN_FAST_INSTALL -pl '!presto-docs,!presto-server-rpm' fi - | if [[ -v HIVE_TESTS ]]; then - ./mvnw install $MAVEN_FAST_INSTALL -P !twitter-modules -pl presto-hive-hadoop2 -am + ./mvnw install $MAVEN_FAST_INSTALL -pl presto-hive-hadoop2 -am fi - | if [[ -v KUDU_TESTS ]]; then @@ -93,15 +93,15 @@ before_script: script: - | if [[ -v MAVEN_CHECKS ]]; then - ./mvnw install -DskipTests -B -T C1 -P 'ci,!twitter-modules' + ./mvnw install -DskipTests -B -T C1 -P ci fi - | if [[ -v TEST_SPECIFIC_MODULES ]]; then - travis_wait 60 ./mvnw test $MAVEN_SKIP_CHECKS_AND_DOCS -P !twitter-modules -B -pl $TEST_SPECIFIC_MODULES $TEST_FLAGS + travis_wait 60 ./mvnw test $MAVEN_SKIP_CHECKS_AND_DOCS -B -pl $TEST_SPECIFIC_MODULES $TEST_FLAGS fi - | if [[ -v TEST_OTHER_MODULES ]]; then - ./mvnw test $MAVEN_SKIP_CHECKS_AND_DOCS -P !twitter-modules -B -pl $TEST_OTHER_MODULES + ./mvnw test $MAVEN_SKIP_CHECKS_AND_DOCS -B -pl $TEST_OTHER_MODULES fi - | if [[ -v PRODUCT_TESTS_BASIC_ENVIRONMENT ]]; then @@ -149,7 +149,7 @@ script: - | if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_2 ]]; then presto-product-tests/bin/run_on_docker.sh \ - multinode-tls -g smoke,group-by,join,tls + multinode-tls -g smoke,cli,group-by,join,tls fi - | if [[ -v PRODUCT_TESTS_SPECIFIC_ENVIRONMENT_2 ]]; then diff --git a/pom.xml b/pom.xml index 1bfc14336bc5..ea502f4fc689 100644 --- a/pom.xml +++ b/pom.xml @@ -123,6 +123,8 @@ presto-proxy presto-kudu twitter-http-client + twitter-eventlistener-plugin + presto-twitter-server @@ -1297,16 +1299,6 @@ - - twitter-modules - - true - - - twitter-eventlistener-plugin - presto-twitter-server - - tests-with-dependencies - - com.twitter - presto-thrift-java - 0.0.5 - - - com.twitter - util-core_2.11 - - - com.twitter - util-core-java - - - com.twitter - util-function_2.10 - - - com.twitter - util-function-java - - - commons-logging - commons-logging - - - org.scala-lang.modules - scala-parser-combinators_2.11 - - - com.twitter - scrooge-core - - - org.scala-lang - scala-library - - - org.scala-lang - scala-reflect - - - - com.twitter util-logging_2.10 @@ -157,12 +113,6 @@ org.scala-lang scala-library 2.10.6 - - - commons-logging - commons-logging - - @@ -178,6 +128,30 @@ test + + io.airlift + http-server + test + + + + io.airlift + jaxrs + test + + + + io.airlift + node + test + + + + javax.ws.rs + javax.ws.rs-api + test + + com.facebook.presto presto-tests @@ -196,4 +170,33 @@ test + + + + + org.apache.thrift.tools + maven-thrift-plugin + 0.1.11 + + /usr/local/bin/thrift + + + + thrift-sources + generate-sources + + compile + + + + thrift-test-sources + generate-test-sources + + testCompile + + + + + + diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/scriber/QueryCompletedEventScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/scriber/QueryCompletedEventScriber.java index afd2230ec3c1..b3b8aebc241a 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/scriber/QueryCompletedEventScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/scriber/QueryCompletedEventScriber.java @@ -51,6 +51,11 @@ public QueryCompletedEventScriber(TwitterEventListenerConfig config) this.scriber = new TwitterScriber(config.getScribeCategory()); } + public QueryCompletedEventScriber(TwitterScriber scriber) + { + this.scriber = requireNonNull(scriber, "scriber is null"); + } + @Override public void handleQueryCompleted(QueryCompletedEvent event) { diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/scriber/TwitterScriber.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/scriber/TwitterScriber.java index 225d0c16d25e..c53e1d668d69 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/scriber/TwitterScriber.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/scriber/TwitterScriber.java @@ -71,7 +71,7 @@ private String serializeThriftToString(TBase thriftMessage) return Base64.getEncoder().encodeToString(serializer.get().serialize(thriftMessage)); } - private void scribe(String message) + protected void scribe(String message) { LogRecord logRecord = new LogRecord(Level.ALL, message); queueingHandler.publish(logRecord); diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBot.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBot.java index 93a9ecf95474..84bef01a4c93 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBot.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBot.java @@ -197,18 +197,26 @@ private void shouldSend(SlackImOpenResponse response, Optional latest, S postMessage.accept(response); return; } + Optional decision = Optional.empty(); + Optional decisionTs = Optional.empty(); for (SlackMessage message : history.getMessages().get()) { Optional result = shouldSend(message, event, principal, state); if (result.isPresent()) { - if (result.get()) { - postMessage.accept(response); + if (!decisionTs.isPresent() || Double.valueOf(decisionTs.get()) < Double.valueOf(message.getTs())) { + decision = result; + decisionTs = Optional.of(message.getTs()); } - return; } if (!newLatest.isPresent() || Double.valueOf(newLatest.get()) > Double.valueOf(message.getTs())) { newLatest = Optional.of(message.getTs()); } } + if (decision.isPresent()) { + if (decision.get()) { + postMessage.accept(response); + } + return; + } if (!history.getHasMore().isPresent() || !history.getHasMore().get()) { postMessage.accept(response); return; @@ -359,7 +367,7 @@ private static T parse(byte[] json, Class javaType) return mapper.readValue(json, javaType); } catch (IOException e) { - throw new IllegalArgumentException(format("Invalid JSON string for %s", javaType), e); + throw new IllegalArgumentException(format("Invalid JSON string [%s] for %s", new String(json), javaType), e); } } diff --git a/twitter-eventlistener-plugin/src/main/thrift/presto.thrift b/twitter-eventlistener-plugin/src/main/thrift/presto.thrift new file mode 100644 index 000000000000..a1132fa9defd --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/thrift/presto.thrift @@ -0,0 +1,112 @@ +namespace java com.twitter.presto.thriftjava +#@namespace scala com.twitter.presto.thriftscala + +enum QueryState { + QUEUED = 1, + PLANNING = 2, + STARTING = 3, + RUNNING = 4, + FINISHING = 5, + FINISHED = 6, + FAILED = 7 +} + +struct OperatorStats { + 1: required i32 pipeline_id + 2: required i32 operator_id + 3: required string plan_node_id + 4: required string operator_type + 5: required i64 total_drivers + 6: required i64 add_input_calls + 7: required i64 add_input_wall_millis + 8: required i64 add_input_cpu_millis + 9: required i64 add_input_user_millis + 10: required i64 input_data_size_bytes + 11: required i64 input_positions + 12: required double sum_squared_input_positions + 13: required i64 get_output_calls + 14: required i64 get_output_wall_millis + 15: required i64 get_output_cpu_millis + 16: required i64 get_output_user_millis + 17: required i64 output_data_size_bytes + 18: required i64 output_positions + 19: required i64 blocked_wall_millis + 20: required i64 finish_calls + 21: required i64 finish_wall_millis + 22: required i64 finish_cpu_millis + 23: required i64 finish_user_millis + 24: required i64 memory_reservation_bytes + 25: required i64 system_memory_reservation_bytes +}(persisted='true') + +struct QueryStageInfo { + 1: required i32 stage_id + 2: required i64 raw_input_data_size_bytes + 3: required i64 output_data_size_bytes + 4: required i32 completed_tasks + 5: required i32 completed_drivers + 6: required double cumulative_memory + 7: required i64 peak_memory_reservation_bytes + 8: required i64 total_scheduled_time_millis + 9: required i64 total_cpu_time_millis + 10: required i64 total_user_time_millis + 11: required i64 total_blocked_time_millis +}(persisted='true') + +/** + * Thrift version of a Presto QueryCompletionEvent. See: + * https://github.com/prestodb/presto/blob/master/presto-main/src/main/java/com/facebook/presto/event/query/QueryCompletionEvent.java + */ +struct QueryCompletionEvent { + 1: required string query_id + 2: optional string transaction_id + 3: required string user + 4: optional string principal + 5: optional string source + 6: optional string server_version + 7: optional string environment + 8: optional string catalog + 9: optional string schema + 10: optional string remote_client_address + 11: optional string user_agent + 12: required QueryState query_state + 13: optional string uri + 14: optional list field_names + 15: required string query + 16: required i64 create_time_ms + 17: required i64 execution_start_time_ms + 18: required i64 end_time_ms + 19: required i64 queued_time_ms + 20: optional i64 analysis_time_ms + 21: required i64 distributed_planning_time_ms + 22: required i64 total_split_wall_time_ms + 23: required i64 total_split_cpu_time_ms + 24: required i64 total_bytes + 25: required i64 total_rows + 26: required i32 splits + 27: optional i32 error_code_id + 28: optional string error_code_name + 29: optional string failure_type + 30: optional string failure_message + 31: optional string failure_task + 32: optional string failure_host + 33: optional string output_stage_json + 34: optional string failures_json + 35: optional string inputs_json + 36: optional string session_properties_json + + # precalcuate some derived data to simplify queries + 200: required i64 query_wall_time_ms + 201: required i64 bytes_per_sec + 202: required i64 bytes_per_cpu_sec + 203: required i64 rows_per_sec + 204: required i64 rows_per_cpu_sec + + 205: optional map> queried_columns_by_table + 206: optional map query_stages + 207: optional list operator_summaries + + 208: optional i64 peak_memory_bytes + 209: optional double cumulative_memory_bytesecond + 210: optional i64 cpu_time_ms +}(persisted='true') diff --git a/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/TestingTwitterEventListenerPlugin.java b/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/TestingTwitterEventListenerPlugin.java index c181dd601421..4ad2d42f08d4 100644 --- a/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/TestingTwitterEventListenerPlugin.java +++ b/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/TestingTwitterEventListenerPlugin.java @@ -21,6 +21,7 @@ import java.util.Map; +import static java.lang.String.format; import static java.util.Objects.requireNonNull; public class TestingTwitterEventListenerPlugin @@ -28,9 +29,9 @@ public class TestingTwitterEventListenerPlugin { private EventListenerFactory factory; - public TestingTwitterEventListenerPlugin(TwitterEventHandler handler) + public TestingTwitterEventListenerPlugin(TwitterEventHandler... handlers) { - this.factory = new TestingTwitterEventListenerFactory(requireNonNull(handler, "handler is null")); + this.factory = new TestingTwitterEventListenerFactory(requireNonNull(handlers, "handler is null")); } @Override @@ -42,11 +43,14 @@ public Iterable getEventListenerFactories() private class TestingTwitterEventListenerFactory implements EventListenerFactory { - private TwitterEventHandler handler; + private TwitterEventHandler[] handlers; - public TestingTwitterEventListenerFactory(TwitterEventHandler handler) + public TestingTwitterEventListenerFactory(TwitterEventHandler... handlers) { - this.handler = requireNonNull(handler, "handler is null"); + for (TwitterEventHandler handler : handlers) { + requireNonNull(handler, format("handler is null")); + } + this.handlers = handlers; } @Override @@ -58,7 +62,7 @@ public String getName() @Override public EventListener create(Map config) { - return new TwitterEventListener(ImmutableSet.of(handler)); + return new TwitterEventListener(ImmutableSet.copyOf(handlers)); } } } diff --git a/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/scriber/TestQueryCompletedEventScriber.java b/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/scriber/TestQueryCompletedEventScriber.java new file mode 100644 index 000000000000..fee7cf6a23fa --- /dev/null +++ b/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/scriber/TestQueryCompletedEventScriber.java @@ -0,0 +1,209 @@ +/* + * 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.twitter.presto.plugin.eventlistener.scriber; + +import com.facebook.presto.Session; +import com.facebook.presto.spi.security.Identity; +import com.facebook.presto.testing.MaterializedResult; +import com.facebook.presto.tests.DistributedQueryRunner; +import com.facebook.presto.tpch.TpchPlugin; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.twitter.presto.plugin.eventlistener.TestingTwitterEventListenerPlugin; +import com.twitter.presto.plugin.eventlistener.TwitterEventHandler; +import com.twitter.presto.thriftjava.QueryCompletionEvent; +import com.twitter.presto.thriftjava.QueryStageInfo; +import com.twitter.presto.thriftjava.QueryState; +import org.apache.thrift.TDeserializer; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.util.ArrayList; +import java.util.Base64; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static com.facebook.presto.testing.TestingSession.testSessionBuilder; +import static com.google.common.collect.Iterables.getOnlyElement; +import static org.testng.Assert.assertEquals; + +@Test(singleThreaded = true) +public class TestQueryCompletedEventScriber +{ + private static final TDeserializer tDeserializer = new TDeserializer(); + // Currently, there is no way to pass principal from test client. + private static final Identity identity = new Identity("test_user", Optional.empty()); + private final TestingTwitterScriber scriber = new TestingTwitterScriber(); + private final TwitterEventHandler handler = new QueryCompletedEventScriber(scriber); + + private DistributedQueryRunner queryRunner; + private Session session; + + @BeforeClass + private void setUp() + throws Exception + { + session = testSessionBuilder() + .setSystemProperty("task_concurrency", "1") + .setCatalog("tpch") + .setSchema("tiny") + .setClientInfo("{\"clientVersion\":\"testVersion\"}") + .build(); + queryRunner = new DistributedQueryRunner(session, 1); + queryRunner.installPlugin(new TpchPlugin()); + queryRunner.installPlugin(new TestingTwitterEventListenerPlugin(handler)); + queryRunner.createCatalog("tpch", "tpch"); + } + + @AfterClass(alwaysRun = true) + private void tearDown() + { + queryRunner.close(); + queryRunner = null; + } + + @Test + public void testConstantQuery() + throws Exception + { + Session session = testSessionBuilder() + .setCatalog("tpch") + .setSchema("tiny") + .setIdentity(identity) + .build(); + runQueryAndWaitForEvents(session, "SELECT 1", 1); + + String queryCompletedEvent = getOnlyElement(scriber.getMessages()); + QueryCompletionEvent tEvent = new QueryCompletionEvent(); + tDeserializer.deserialize(tEvent, Base64.getDecoder().decode(queryCompletedEvent)); + + // check user audit information + assertEquals(tEvent.getUser(), identity.getUser()); + + // check server audit information + assertEquals(tEvent.getSource(), "test"); + assertEquals(tEvent.getServer_version(), "testversion"); + assertEquals(tEvent.getEnvironment(), "testing"); + + // check query audit information + assertEquals(tEvent.getCatalog(), "tpch"); + assertEquals(tEvent.getSchema(), "tiny"); + assertEquals(tEvent.getQuery(), "SELECT 1"); + assertEquals(tEvent.getQuery_state(), QueryState.FINISHED); + + // check query stats information + assertEquals(tEvent.getTotal_rows(), 0L); + assertEquals(tEvent.getTotal_bytes(), 0L); + } + + @Test + public void testNormalQuery() + throws Exception + { + runQueryAndWaitForEvents(session, "SELECT sum(linenumber) FROM lineitem", 1); + + String queryCompletedEvent = getOnlyElement(scriber.getMessages()); + QueryCompletionEvent tEvent = new QueryCompletionEvent(); + tDeserializer.deserialize(tEvent, Base64.getDecoder().decode(queryCompletedEvent)); + + // check query audit information + assertEquals(tEvent.getQueried_columns_by_table(), ImmutableMap.of("sf0.01.lineitem", ImmutableList.of("linenumber"))); + assertEquals(tEvent.getQuery_state(), QueryState.FINISHED); + + // check query state information + assertEquals(tEvent.getTotal_rows(), 60175L); + assertEquals(tEvent.getTotal_bytes(), 0L); + assertEquals(tEvent.getSplits(), tEvent.getQuery_stages().values().stream() + .map(QueryStageInfo::getCompleted_drivers) + .reduce(0, Integer::sum).intValue()); + assertEquals(tEvent.getCpu_time_ms(), tEvent.getQuery_stages().values().stream() + .map(QueryStageInfo::getTotal_cpu_time_millis) + .reduce(0L, Long::sum).longValue()); + } + + @Test + public void testFailedQuery() + throws Exception + { + String failureMessage; + // Run a query with Syntax Error + try { + runQueryAndWaitForEvents(session, "SELECT notexistcolumn FROM lineitem", 1); + failureMessage = ""; + } + catch (Exception e) { + failureMessage = e.getMessage(); + } + + String queryCompletedEvent = getOnlyElement(scriber.getMessages()); + QueryCompletionEvent tEvent = new QueryCompletionEvent(); + tDeserializer.deserialize(tEvent, Base64.getDecoder().decode(queryCompletedEvent)); + + // check query failure information + assertEquals(tEvent.getQuery_state(), QueryState.FAILED); + assertEquals(tEvent.getError_code_id(), 1); + assertEquals(tEvent.getError_code_name(), "SYNTAX_ERROR"); + assertEquals(tEvent.getFailure_message(), failureMessage); + } + + private MaterializedResult runQueryAndWaitForEvents(Session session, String sql, int numMessagesExpected) + throws Exception + { + scriber.initialize(numMessagesExpected); + MaterializedResult result = queryRunner.execute(session, sql); + scriber.waitForEvents(10); + + return result; + } + + private class TestingTwitterScriber + extends TwitterScriber + { + private final List messages; + private CountDownLatch messagesLatch; + + public TestingTwitterScriber() + { + super("test"); + this.messages = new ArrayList<>(); + } + + public synchronized void initialize(int numEvents) + { + messagesLatch = new CountDownLatch(numEvents); + messages.clear(); + } + + public void waitForEvents(int timeoutSeconds) + throws InterruptedException + { + messagesLatch.await(timeoutSeconds, TimeUnit.SECONDS); + } + + public List getMessages() + { + return ImmutableList.copyOf(messages); + } + + @Override + protected void scribe(String message) + { + messages.add(message); + messagesLatch.countDown(); + } + } +} diff --git a/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/slack/TestSlackBot.java b/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/slack/TestSlackBot.java new file mode 100644 index 000000000000..d38b54b05a98 --- /dev/null +++ b/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/slack/TestSlackBot.java @@ -0,0 +1,274 @@ +/* + * 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.twitter.presto.plugin.eventlistener.slack; + +import com.facebook.presto.Session; +import com.facebook.presto.testing.MaterializedResult; +import com.facebook.presto.tests.DistributedQueryRunner; +import com.facebook.presto.tpch.TpchPlugin; +import com.google.common.collect.ImmutableList; +import com.google.inject.Injector; +import com.twitter.presto.plugin.eventlistener.TestingTwitterEventListenerPlugin; +import com.twitter.presto.plugin.eventlistener.TwitterEventHandler; +import com.twitter.presto.plugin.eventlistener.TwitterEventListenerConfig; +import io.airlift.bootstrap.Bootstrap; +import io.airlift.bootstrap.LifeCycleManager; +import io.airlift.http.server.testing.TestingHttpServer; +import io.airlift.http.server.testing.TestingHttpServerModule; +import io.airlift.jaxrs.JaxrsModule; +import io.airlift.json.JsonModule; +import io.airlift.log.Logger; +import io.airlift.node.testing.TestingNodeModule; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.net.URI; +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static com.facebook.presto.testing.TestingSession.testSessionBuilder; +import static io.airlift.jaxrs.JaxrsBinder.jaxrsBinder; +import static io.airlift.testing.Assertions.assertEqualsIgnoreOrder; + +@Test(singleThreaded = true) +public class TestSlackBot +{ + private static Logger log = Logger.get(TestSlackBot.class); + private static final Optional CREATED = Optional.of("Hi there, I just started a new query."); + private static final Optional FINISHED = Optional.of("I just completed your query."); + private static final Optional FAILED = Optional.of("Unfortunately, your query was failed due to error: ${FAILURE_MESSAGE}"); + + private Map imOpenResponses = new HashMap<>(); + private Map, SlackChatPostMessageResponse> chatPostMessageResponses = new HashMap<>(); + private Map usersLookupByEmailResponses = new HashMap<>(); + private Map, SlackImHistoryResponse> imHistoryResponses = new HashMap<>(); + + private DistributedQueryRunner queryRunner; + private Session session; + + private TestingSlackResource resource; + private LifeCycleManager lifeCycleManager; + private TestingHttpServer server; + + @BeforeClass + private void setUp() + throws Exception + { + Bootstrap app = new Bootstrap( + new TestingNodeModule(), + new TestingHttpServerModule(), + new JsonModule(), + new JaxrsModule(true), + binder -> jaxrsBinder(binder).bind(TestingSlackResource.class)); + + Injector injector = app + .strictConfig() + .doNotInitializeLogging() + .initialize(); + + lifeCycleManager = injector.getInstance(LifeCycleManager.class); + server = injector.getInstance(TestingHttpServer.class); + resource = injector.getInstance(TestingSlackResource.class); + resource.initialize(imOpenResponses, chatPostMessageResponses, usersLookupByEmailResponses, imHistoryResponses); + + session = testSessionBuilder() + .setSystemProperty("task_concurrency", "1") + .setCatalog("tpch") + .setSchema("tiny") + .setClientInfo("{\"clientVersion\":\"testVersion\"}") + .build(); + + TwitterEventListenerConfig config = new TwitterEventListenerConfig() + .setSlackConfigFile(getResourceFilePath("slackCredentials.json")) + .setSlackUri(uriFor("")) + .setSlackEmailTemplate("${USER}@example.com") + .setSlackNotificationTemplateFile(getResourceFilePath("slackNotifications.json")) + .setSlackUsers("user"); + TwitterEventHandler handler = new SlackBot(config); + + queryRunner = new DistributedQueryRunner(session, 1); + queryRunner.installPlugin(new TpchPlugin()); + queryRunner.installPlugin(new TestingTwitterEventListenerPlugin(handler)); + queryRunner.createCatalog("tpch", "tpch"); + } + + @AfterClass(alwaysRun = true) + private void tearDown() + throws Exception + { + try { + lifeCycleManager.stop(); + } + catch (Exception e) { + // ignore + } + finally { + lifeCycleManager = null; + } + + try { + queryRunner.close(); + } + catch (Exception e) { + // ignore + } + finally { + queryRunner = null; + } + } + + @Test + public void testConstantQuery() + throws Exception + { + clearSlackResource(); + prepareSlackResource("user@example.com", "channel_id", "user_id", "user_name", CREATED, FINISHED, FAILED, Optional.empty()); + + runQueryAndWaitForEvents(session, "SELECT 1", 8); + + assertEqualsIgnoreOrder(resource.getUsersLookupByEmailRequests(), ImmutableList.of("user@example.com", "user@example.com")); + assertEqualsIgnoreOrder(resource.getImOpenRequests(), ImmutableList.of("user_id", "user_id")); + assertEqualsIgnoreOrder(resource.getImHistoryRequests(), ImmutableList.of(getEntry("channel_id", null), getEntry("channel_id", null))); + assertEqualsIgnoreOrder(resource.getChatPostMessageRequests(), ImmutableList.of(getEntry("channel_id", CREATED.get()), getEntry("channel_id", FINISHED.get()))); + } + + @Test + public void testPartialStoppedChannel() + throws Exception + { + clearSlackResource(); + List channelHistory = new ArrayList<>(); + channelHistory.add(new SlackMessage("stop event=created", "0")); + prepareSlackResource("user@example.com", "channel_id", "user_id", "user_name", CREATED, FINISHED, FAILED, Optional.of(channelHistory)); + + runQueryAndWaitForEvents(session, "SELECT 1", 7); + + assertEqualsIgnoreOrder(resource.getUsersLookupByEmailRequests(), ImmutableList.of("user@example.com", "user@example.com")); + assertEqualsIgnoreOrder(resource.getImOpenRequests(), ImmutableList.of("user_id", "user_id")); + assertEqualsIgnoreOrder(resource.getImHistoryRequests(), ImmutableList.of(getEntry("channel_id", null), getEntry("channel_id", null))); + assertEqualsIgnoreOrder(resource.getChatPostMessageRequests(), ImmutableList.of(getEntry("channel_id", FINISHED.get()))); + } + + @Test + public void testResumedChannel() + throws Exception + { + clearSlackResource(); + + List channelHistory = new ArrayList<>(); + channelHistory.add(new SlackMessage("resume event=created", "1")); + channelHistory.add(new SlackMessage("stop event=created", "0")); + + prepareSlackResource("user@example.com", "channel_id", "user_id", "user_name", CREATED, FINISHED, FAILED, Optional.of(channelHistory)); + + runQueryAndWaitForEvents(session, "SELECT 1", 8); + + assertEqualsIgnoreOrder(resource.getUsersLookupByEmailRequests(), ImmutableList.of("user@example.com", "user@example.com")); + assertEqualsIgnoreOrder(resource.getImOpenRequests(), ImmutableList.of("user_id", "user_id")); + assertEqualsIgnoreOrder(resource.getImHistoryRequests(), ImmutableList.of(getEntry("channel_id", null), getEntry("channel_id", null))); + assertEqualsIgnoreOrder(resource.getChatPostMessageRequests(), ImmutableList.of(getEntry("channel_id", CREATED.get()), getEntry("channel_id", FINISHED.get()))); + } + + @Test + public void testFailedQuery() + throws Exception + { + clearSlackResource(); + prepareSlackResource("user@example.com", "channel_id", "user_id", "user_name", CREATED, Optional.empty(), Optional.empty(), Optional.empty()); + allowWildCardChatPostMessage(); + + String failureMessage; + // Run a query with Syntax Error + try { + runQueryAndWaitForEvents(session, "SELECT notexistcolumn FROM lineitem", 8); + failureMessage = ""; + } + catch (Exception e) { + failureMessage = e.getMessage(); + } + finally { + // make sure it consumed all slack requests + resource.waitForCalls(20); + } + + assertEqualsIgnoreOrder(resource.getUsersLookupByEmailRequests(), ImmutableList.of("user@example.com", "user@example.com")); + assertEqualsIgnoreOrder(resource.getImOpenRequests(), ImmutableList.of("user_id", "user_id")); + assertEqualsIgnoreOrder(resource.getImHistoryRequests(), ImmutableList.of(getEntry("channel_id", null), getEntry("channel_id", null))); + assertEqualsIgnoreOrder(resource.getChatPostMessageRequests(), ImmutableList.of(getEntry("channel_id", CREATED.get()), getEntry("channel_id", FAILED.get().replaceAll("\\$\\{FAILURE_MESSAGE}", failureMessage)))); + } + + private MaterializedResult runQueryAndWaitForEvents(Session session, String sql, int numCallsExpected) + throws Exception + { + resource.setNumCallsExpected(numCallsExpected); + MaterializedResult result = queryRunner.execute(session, sql); + resource.waitForCalls(20); + + return result; + } + + private void clearSlackResource() + { + usersLookupByEmailResponses.clear(); + imOpenResponses.clear(); + imHistoryResponses.clear(); + chatPostMessageResponses.clear(); + } + + private void prepareSlackResource( + String email, + String channel, + String userId, + String realName, + Optional createdMessage, + Optional finishedMessage, + Optional failedMessage, + Optional> historyMessages) + { + usersLookupByEmailResponses.put(email, new SlackUsersLookupByEmailResponse(true, Optional.of(new SlackUser(userId, realName)), Optional.empty())); + imOpenResponses.put(userId, new SlackImOpenResponse(true, Optional.of(new SlackChannel(channel)), Optional.empty())); + + SlackChatPostMessageResponse simpleOk = new SlackChatPostMessageResponse(true, Optional.empty(), Optional.empty()); + createdMessage.ifPresent(message -> chatPostMessageResponses.put(getEntry(channel, message), simpleOk)); + finishedMessage.ifPresent(message -> chatPostMessageResponses.put(getEntry(channel, message), simpleOk)); + failedMessage.ifPresent(message -> chatPostMessageResponses.put(getEntry(channel, message), simpleOk)); + + imHistoryResponses.put(getEntry(channel, null), new SlackImHistoryResponse(true, Optional.empty(), historyMessages, Optional.of(false), Optional.empty())); + } + + private void allowWildCardChatPostMessage() + { + chatPostMessageResponses.put(getEntry("*", "*"), new SlackChatPostMessageResponse(true, Optional.empty(), Optional.empty())); + } + + private Map.Entry getEntry(String key, String value) + { + return new AbstractMap.SimpleImmutableEntry<>(key, value); + } + + private URI uriFor(String path) + { + return server.getBaseUrl().resolve(path); + } + + private String getResourceFilePath(String fileName) + { + return this.getClass().getClassLoader().getResource(fileName).getPath(); + } +} diff --git a/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/slack/TestingSlackResource.java b/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/slack/TestingSlackResource.java new file mode 100644 index 000000000000..974ced63bae8 --- /dev/null +++ b/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/slack/TestingSlackResource.java @@ -0,0 +1,163 @@ +/* + * 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.twitter.presto.plugin.eventlistener.slack; + +import javax.ws.rs.Consumes; +import javax.ws.rs.FormParam; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; + +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static java.util.Objects.requireNonNull; + +@Path("/api") +public class TestingSlackResource +{ + private Map imOpenResponses; + private Map, SlackChatPostMessageResponse> chatPostMessageResponses; + private Map usersLookupByEmailResponses; + private Map, SlackImHistoryResponse> imHistoryResponses; + + private List imOpenRequests = new ArrayList<>(); + private List> chatPostMessageRequests = new ArrayList<>(); + private List usersLookupByEmailRequests = new ArrayList<>(); + private List> imHistoryRequests = new ArrayList<>(); + + private CountDownLatch numCallsExpected; + + public void initialize( + Map imOpenResponses, + Map, SlackChatPostMessageResponse> chatPostMessageResponses, + Map usersLookupByEmailResponses, + Map, SlackImHistoryResponse> imHistoryResponses) + { + this.imOpenResponses = requireNonNull(imOpenResponses, "imOpenResponses is null"); + this.chatPostMessageResponses = requireNonNull(chatPostMessageResponses, "chatPostMessageResponses is null"); + this.usersLookupByEmailResponses = requireNonNull(usersLookupByEmailResponses, "usersLookupByEmailResponses is null"); + this.imHistoryResponses = requireNonNull(imHistoryResponses, "imHistoryResponses is null"); + } + + public void setNumCallsExpected(int numCallsExpected) + { + this.numCallsExpected = new CountDownLatch(numCallsExpected); + imOpenRequests.clear(); + chatPostMessageRequests.clear(); + usersLookupByEmailRequests.clear(); + imHistoryRequests.clear(); + } + + public void waitForCalls(int timeoutSeconds) + throws InterruptedException + { + numCallsExpected.await(timeoutSeconds, TimeUnit.SECONDS); + } + + public List getImOpenRequests() + { + return imOpenRequests; + } + + public List> getChatPostMessageRequests() + { + return chatPostMessageRequests; + } + + public List getUsersLookupByEmailRequests() + { + return usersLookupByEmailRequests; + } + + public List> getImHistoryRequests() + { + return imHistoryRequests; + } + + @Path("/im.open") + @POST + @Consumes(MediaType.APPLICATION_JSON) + public Response ImOpen(SlackImOpenRequest request) + { + imOpenRequests.add(request.getUser()); + numCallsExpected.countDown(); + SlackImOpenResponse response = imOpenResponses.get(request.getUser()); + if (response == null) { + response = imOpenResponses.get("*"); + } + if (response == null) { + return Response.status(Response.Status.BAD_REQUEST).build(); + } + return Response.ok(response, MediaType.APPLICATION_JSON_TYPE).build(); + } + + @Path("/chat.postMessage") + @POST + @Consumes(MediaType.APPLICATION_JSON) + public Response ChatPostMessage(SlackChatPostMessageRequest request) + { + Map.Entry entry = new AbstractMap.SimpleImmutableEntry<>(request.getChannel(), request.getText()); + chatPostMessageRequests.add(entry); + numCallsExpected.countDown(); + SlackChatPostMessageResponse response = chatPostMessageResponses.get(entry); + if (response == null) { + response = chatPostMessageResponses.get(new AbstractMap.SimpleImmutableEntry<>("*", "*")); + } + if (response == null) { + return Response.status(Response.Status.BAD_REQUEST).build(); + } + return Response.ok(response, MediaType.APPLICATION_JSON_TYPE).build(); + } + + @Path("/users.lookupByEmail") + @POST + @Consumes(MediaType.APPLICATION_FORM_URLENCODED) + public Response UsersLookupByEmail(@FormParam("email") String email) + { + usersLookupByEmailRequests.add(email); + numCallsExpected.countDown(); + SlackUsersLookupByEmailResponse response = usersLookupByEmailResponses.get(email); + if (response == null) { + response = usersLookupByEmailResponses.get("*"); + } + if (response == null) { + return Response.status(Response.Status.BAD_REQUEST).build(); + } + return Response.ok(response, MediaType.APPLICATION_JSON_TYPE).build(); + } + + @Path("/im.history") + @POST + @Consumes(MediaType.APPLICATION_FORM_URLENCODED) + public Response ImHistory(@FormParam("channel") String channel, @FormParam("latest") String latest) + { + Map.Entry entry = new AbstractMap.SimpleImmutableEntry<>(channel, latest); + imHistoryRequests.add(entry); + numCallsExpected.countDown(); + SlackImHistoryResponse response = imHistoryResponses.get(entry); + if (response == null) { + response = imHistoryResponses.get(new AbstractMap.SimpleImmutableEntry<>("*", "*")); + } + if (response == null) { + return Response.status(Response.Status.BAD_REQUEST).build(); + } + return Response.ok(response, MediaType.APPLICATION_JSON_TYPE).build(); + } +} diff --git a/twitter-eventlistener-plugin/src/test/resources/slackCredentials.json b/twitter-eventlistener-plugin/src/test/resources/slackCredentials.json new file mode 100644 index 000000000000..3a8ad8b910f6 --- /dev/null +++ b/twitter-eventlistener-plugin/src/test/resources/slackCredentials.json @@ -0,0 +1,3 @@ +{ + "token": "test_token" +} diff --git a/twitter-eventlistener-plugin/src/test/resources/slackNotifications.json b/twitter-eventlistener-plugin/src/test/resources/slackNotifications.json new file mode 100644 index 000000000000..cb4a7e077931 --- /dev/null +++ b/twitter-eventlistener-plugin/src/test/resources/slackNotifications.json @@ -0,0 +1,18 @@ +{ + "templates": [ + { + "event": "created", + "text": "Hi there, I just started a new query." + }, + { + "event": "completed", + "state": "FINISHED", + "text": "I just completed your query." + }, + { + "event": "completed", + "state": "FAILED", + "text": "Unfortunately, your query was failed due to error: ${FAILURE_MESSAGE}" + } + ] +} From ccf33254e065024f2c64950716aa7ab9bc970b43 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 20 Nov 2018 19:44:47 -0800 Subject: [PATCH 315/331] Release minor revision 0.210-tw-0.56 (#183) --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial-toolkit/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kudu/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory-context/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-password-authenticators/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-proxy/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 2 +- twitter-http-client/pom.xml | 2 +- 60 files changed, 61 insertions(+), 61 deletions(-) diff --git a/pom.xml b/pom.xml index ea502f4fc689..8b1368d6bca5 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.210-tw-0.55 + 0.210-tw-0.56 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 34453256f28e..97c6dacec422 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index dbc1c61d1242..52e0c82b85d0 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 2b573d676c5a..e7a0e7f4ded2 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 6df2d874526c..bd88ab457c10 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 0548b7a47de8..196344ec8aea 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 78ad5af35d58..234a858e0d1a 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.55 + 0.210-tw-0.56 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 692a76156f42..0b753b471859 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 75f9f6efe990..8b29c27197e9 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index bc0b1aeb6c9a..eb0c38d47168 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 44664fd28700..8ab71418829e 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 64c22f4a0864..1e0eb07ff3ac 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 3a6ac2c520a5..3a9c2a367017 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 0d91f6862b14..58fbf5850391 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index 47a9b095269b..52186e544214 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index 1b67c90c483a..c52305a6d5c7 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 53ecd60b6090..7d775c6f24dd 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index eb863f53027a..d2ac7f53fa87 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index eae0ec10ba98..61086a8cb929 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 0459524f1391..95878b152bac 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index a0aa4bf326c0..58c644ddbee8 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-kafka diff --git a/presto-kudu/pom.xml b/presto-kudu/pom.xml index 15702c14833b..bf484a69bffe 100644 --- a/presto-kudu/pom.xml +++ b/presto-kudu/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-kudu diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index ae95b1d2a403..649c7d5dc586 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 1a0e44caf41e..76d78e17f1bc 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index 2f9223efd378..f0f2ec80b533 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.210-tw-0.55 + 0.210-tw-0.56 presto-matching diff --git a/presto-memory-context/pom.xml b/presto-memory-context/pom.xml index b85c292c73a0..e1ec0c863ff2 100644 --- a/presto-memory-context/pom.xml +++ b/presto-memory-context/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-memory-context diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 14ce9dc2bfd9..47a60e8b4999 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 11944938921d..8266e5a6d931 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 81ab3246b520..474d07731570 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 1d91d0255e42..5a14900112e5 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 7eff22426adc..d90f67a968de 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 396cfbec7dcc..5c99ce3b5bbd 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-parser diff --git a/presto-password-authenticators/pom.xml b/presto-password-authenticators/pom.xml index 3777f48ce22d..e4cd8947aae6 100644 --- a/presto-password-authenticators/pom.xml +++ b/presto-password-authenticators/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-password-authenticators diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index ea7e4bcbae3e..fa9002883632 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index a8f1a239eaaa..40fc441226e3 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index d40aa9dac9e0..43a4530a1555 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.55 + 0.210-tw-0.56 presto-product-tests diff --git a/presto-proxy/pom.xml b/presto-proxy/pom.xml index 941460638859..e0104398d44a 100644 --- a/presto-proxy/pom.xml +++ b/presto-proxy/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-proxy diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 6508c662cf64..ab6c66032e39 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 5123951ab9ec..5c1b65cf73ba 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 74568ddb6139..be36fd56a990 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index f2a4c3977341..7a683a89f519 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index bfa8171aab26..48a95b3fed96 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index d6a2efd03946..d2fa8e9950b8 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index a48a4e0af2aa..127e4effb767 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index d6c5d420c31b..640e1bfcd9fc 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 83e2f2308ee3..7d0ff16ac773 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 0545e6373a7d..b14e9acbed8e 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.210-tw-0.55 + 0.210-tw-0.56 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 491ce4dc81d1..dbfec3844366 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 0663ea5855f9..a15a6e152274 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index c91a6155dec8..1b13942da6b0 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.55 + 0.210-tw-0.56 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index 05de0e7f5cfb..ee47a072f18b 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index e652d9dbac85..c72271ed20d1 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index 97b06cb62fb4..89649f524fb2 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index ffa8da1e0e1f..6f64bce66949 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 0351dcef2fce..7920550f1d4a 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index b9acecf1d8fc..e4aef010e37a 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-twitter-functions diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index d6e9225e25ed..7d1ea9efff23 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index f1e254868041..8ff4fa13519b 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 968c217a30a7..5ca85fef4985 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.55 + 0.210-tw-0.56 twitter-eventlistener-plugin diff --git a/twitter-http-client/pom.xml b/twitter-http-client/pom.xml index a5ca8bd77489..4af377693446 100644 --- a/twitter-http-client/pom.xml +++ b/twitter-http-client/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.55 + 0.210-tw-0.56 4.0.0 From f321de5ea8dd3053654dd66a9c0f7982019d3257 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 12 Dec 2018 18:16:44 -0800 Subject: [PATCH 316/331] Add KEY_OF_MAX_VALUE map function (#184) --- .../src/main/sphinx/functions/twitter.rst | 7 +++ .../functions/TwitterFunctionsPlugin.java | 1 + .../functions/TwitterMapFunctions.java | 56 +++++++++++++++++++ .../functions/TestTwitterFunctions.java | 12 ++++ 4 files changed, 76 insertions(+) create mode 100644 presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterMapFunctions.java diff --git a/presto-docs/src/main/sphinx/functions/twitter.rst b/presto-docs/src/main/sphinx/functions/twitter.rst index 03bec3f9247d..573bea916b7b 100644 --- a/presto-docs/src/main/sphinx/functions/twitter.rst +++ b/presto-docs/src/main/sphinx/functions/twitter.rst @@ -4,6 +4,13 @@ Twitter Functions These functions provide some convenience functionality commonly used at Twitter. +Map Functions +----------------- + +.. function:: key_of_max_value(map) -> K + + Get the key of the entry of map that holding max value. If more than one entry holds the same max value, return null. + String Functions ----------------- diff --git a/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterFunctionsPlugin.java b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterFunctionsPlugin.java index 6a9a491e9306..8f419d6ea224 100644 --- a/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterFunctionsPlugin.java +++ b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterFunctionsPlugin.java @@ -26,6 +26,7 @@ public Set> getFunctions() { return ImmutableSet.>builder() .add(TwitterStringFunctions.class) + .add(TwitterMapFunctions.class) .add(SnowflakeFunctions.class) .build(); } diff --git a/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterMapFunctions.java b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterMapFunctions.java new file mode 100644 index 000000000000..b7d0c67811a1 --- /dev/null +++ b/presto-twitter-functions/src/main/java/com/facebook/presto/twitter/functions/TwitterMapFunctions.java @@ -0,0 +1,56 @@ +/* + * 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.twitter.functions; + +import com.facebook.presto.spi.block.Block; +import com.facebook.presto.spi.function.Description; +import com.facebook.presto.spi.function.ScalarFunction; +import com.facebook.presto.spi.function.SqlNullable; +import com.facebook.presto.spi.function.SqlType; +import com.facebook.presto.spi.function.TypeParameter; +import com.facebook.presto.spi.type.Type; + +import static com.facebook.presto.spi.type.BigintType.BIGINT; +import static com.facebook.presto.spi.type.TypeUtils.readNativeValue; + +public class TwitterMapFunctions +{ + private TwitterMapFunctions() + { + } + + @ScalarFunction("key_of_max_value") + @Description("Get the key of the entry of map that holding max value. If more than one entry holds the same max value, return null") + @TypeParameter("K") + @SqlType("K") + @SqlNullable + public static Object keyMaxValue(@TypeParameter("K") Type keyType, @SqlType("map(K,bigint)") Block map) + { + Object keyOfMaxValue = null; + long maxValue = Long.MIN_VALUE; + for (int position = 0; position < map.getPositionCount(); position += 2) { + Object key = readNativeValue(keyType, map, position); + long value = BIGINT.getLong(map, position + 1); + if (value > maxValue) { + keyOfMaxValue = key; + maxValue = value; + } + else if (value == maxValue) { + keyOfMaxValue = null; + } + } + + return keyOfMaxValue; + } +} diff --git a/presto-twitter-functions/src/test/java/com/facebook/presto/twitter/functions/TestTwitterFunctions.java b/presto-twitter-functions/src/test/java/com/facebook/presto/twitter/functions/TestTwitterFunctions.java index 4c75f59a88f5..c077692d19d0 100644 --- a/presto-twitter-functions/src/test/java/com/facebook/presto/twitter/functions/TestTwitterFunctions.java +++ b/presto-twitter-functions/src/test/java/com/facebook/presto/twitter/functions/TestTwitterFunctions.java @@ -23,6 +23,7 @@ import static com.facebook.presto.metadata.FunctionExtractor.extractFunctions; import static com.facebook.presto.spi.type.BigintType.BIGINT; import static com.facebook.presto.spi.type.BooleanType.BOOLEAN; +import static com.facebook.presto.spi.type.DoubleType.DOUBLE; import static com.facebook.presto.spi.type.TimeZoneKey.UTC_KEY; import static com.facebook.presto.spi.type.TimestampType.TIMESTAMP; import static com.facebook.presto.spi.type.VarcharType.createVarcharType; @@ -79,4 +80,15 @@ public void testSnowflake() assertFunction("SEQUENCE_NUM_FROM_SNOWFLAKE(265605588183052288)", BIGINT, 0L); assertInvalidFunction("SEQUENCE_NUM_FROM_SNOWFLAKE(1000)", "Not a Snowflake Id: 1000"); } + + @Test + public void testKeyOfMaxValue() + { + assertFunction("KEY_OF_MAX_VALUE(MAP(ARRAY['foo', 'bar'], ARRAY[1, 2]))", createVarcharType(3), "bar"); + assertFunction("KEY_OF_MAX_VALUE(CAST(MAP(ARRAY[100.0, 200.0], ARRAY[1, 2]) AS MAP(DOUBLE, BIGINT)))", DOUBLE, 200.0); + assertFunction("KEY_OF_MAX_VALUE(CAST(MAP(ARRAY[100, 200], ARRAY[1, 2]) AS MAP(BIGINT, BIGINT)))", BIGINT, 200L); + assertFunction("KEY_OF_MAX_VALUE(CAST(MAP(ARRAY[1, 0], ARRAY[1,2]) AS MAP(BOOLEAN, BIGINT)))", BOOLEAN, false); + + assertFunction("KEY_OF_MAX_VALUE(CAST(MAP(ARRAY[1, 0], ARRAY[2,2]) AS MAP(BOOLEAN, BIGINT)))", BOOLEAN, null); + } } From e81547ab3fe8adda31b844d40a2822fe37b33990 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 12 Dec 2018 18:17:28 -0800 Subject: [PATCH 317/331] Fix issue in lzo index re-alignment (#185) --- .../twitter/hive/thrift/ThriftHiveRecordCursorProvider.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) 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 8c10e78f5f33..730c1cd6e30a 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 @@ -91,6 +91,9 @@ public Optional createRecordCursor( setPropertyIfUnset(schema, "elephantbird.mapred.input.bad.record.check.only.in.close", Boolean.toString(false)); setPropertyIfUnset(schema, "elephantbird.mapred.input.bad.record.threshold", Float.toString(0.0f)); + long originalStart = start; + long originalLength = length; + // re-align split range if (isLzopCompressedFile(path)) { LzoIndex index = new LzoIndex(); @@ -120,7 +123,7 @@ public Optional createRecordCursor( else { // if index is empty but start is not NOT_FOUND, read the whole file, otherwise align split end. length = index.isEmpty() ? fileSize : - (Math.min(index.alignSliceEndToIndex(start + length, fileSize), fileSize) - start); + (Math.min(index.alignSliceEndToIndex(originalStart + length, fileSize), fileSize) - start); } } From 4080e2975aa3570322f98132135dfbc961987e63 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 12 Dec 2018 18:35:15 -0800 Subject: [PATCH 318/331] Release minor revision 0.210-tw-0.57 (#186) --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial-toolkit/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kudu/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory-context/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-password-authenticators/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-proxy/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 2 +- twitter-http-client/pom.xml | 2 +- 60 files changed, 61 insertions(+), 61 deletions(-) diff --git a/pom.xml b/pom.xml index 8b1368d6bca5..395564f2f078 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.210-tw-0.56 + 0.210-tw-0.57 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 97c6dacec422..6bda6e40582b 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 52e0c82b85d0..b6b2c397096b 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index e7a0e7f4ded2..aa1069d95319 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index bd88ab457c10..bcd5e534a109 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 196344ec8aea..d30c40cfe9dd 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index 234a858e0d1a..f116fa5ffedf 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.56 + 0.210-tw-0.57 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 0b753b471859..de7606a1703e 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 8b29c27197e9..e17cdafe697a 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index eb0c38d47168..fdadad83589f 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 8ab71418829e..5c30da225c79 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 1e0eb07ff3ac..06796a76eb4d 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 3a9c2a367017..10313397d868 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 58fbf5850391..5ddd4b98e087 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index 52186e544214..c47f1acbf801 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index c52305a6d5c7..a4040b320f51 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 7d775c6f24dd..ce59496c4c69 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index d2ac7f53fa87..5a4a9a839c75 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 61086a8cb929..e8829e08f788 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 95878b152bac..567a1cb01470 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 58c644ddbee8..1ff3f9e72643 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-kafka diff --git a/presto-kudu/pom.xml b/presto-kudu/pom.xml index bf484a69bffe..e3c617c37851 100644 --- a/presto-kudu/pom.xml +++ b/presto-kudu/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-kudu diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 649c7d5dc586..fc649187ac0e 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 76d78e17f1bc..c90677d349d8 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index f0f2ec80b533..5f6901a0d4fe 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.210-tw-0.56 + 0.210-tw-0.57 presto-matching diff --git a/presto-memory-context/pom.xml b/presto-memory-context/pom.xml index e1ec0c863ff2..2a70dd3cd895 100644 --- a/presto-memory-context/pom.xml +++ b/presto-memory-context/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-memory-context diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 47a60e8b4999..373016839a32 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 8266e5a6d931..868eed70c4e6 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 474d07731570..8b444ccec8fd 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 5a14900112e5..2ff1c7ec8b50 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index d90f67a968de..b2936c46b4a2 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 5c99ce3b5bbd..18334bac206e 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-parser diff --git a/presto-password-authenticators/pom.xml b/presto-password-authenticators/pom.xml index e4cd8947aae6..fe3f6b237ce1 100644 --- a/presto-password-authenticators/pom.xml +++ b/presto-password-authenticators/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-password-authenticators diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index fa9002883632..ddebec1db5b5 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 40fc441226e3..d43f0008fb15 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 43a4530a1555..3f8d0921d3f0 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.56 + 0.210-tw-0.57 presto-product-tests diff --git a/presto-proxy/pom.xml b/presto-proxy/pom.xml index e0104398d44a..22d3faca6dcb 100644 --- a/presto-proxy/pom.xml +++ b/presto-proxy/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-proxy diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index ab6c66032e39..90611d9081aa 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 5c1b65cf73ba..cec3cb4bdf3a 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index be36fd56a990..c4428c676d1a 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 7a683a89f519..259afbebe561 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index 48a95b3fed96..90ab5576f94b 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index d2fa8e9950b8..897acecbb519 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 127e4effb767..e28dd842a053 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 640e1bfcd9fc..a1f33205c5ca 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 7d0ff16ac773..1042194da613 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index b14e9acbed8e..8c07fc8a2231 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.210-tw-0.56 + 0.210-tw-0.57 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index dbfec3844366..36a41a60c174 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index a15a6e152274..d5b7e21210f8 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 1b13942da6b0..f54af7df07ad 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.56 + 0.210-tw-0.57 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index ee47a072f18b..b295dc9c0cb9 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index c72271ed20d1..33a21628b559 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index 89649f524fb2..0d1e6f29e317 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index 6f64bce66949..c2eecfbd6b1b 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 7920550f1d4a..6055b966abac 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index e4aef010e37a..93644767f0c5 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-twitter-functions diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index 7d1ea9efff23..cb070d5d6392 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 8ff4fa13519b..d64f4711fc95 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 5ca85fef4985..33224d14b9a0 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.56 + 0.210-tw-0.57 twitter-eventlistener-plugin diff --git a/twitter-http-client/pom.xml b/twitter-http-client/pom.xml index 4af377693446..02f499cb2f1f 100644 --- a/twitter-http-client/pom.xml +++ b/twitter-http-client/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.56 + 0.210-tw-0.57 4.0.0 From 876b71faa14c6eba1a350e142afdac26bd4a09a3 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 8 Jan 2019 14:42:18 -0800 Subject: [PATCH 319/331] Add wall time and error type filters in Slack notification templates (#187) --- .../eventlistener/slack/AbsentOrPattern.java | 62 ++++++++++++++++++ .../plugin/eventlistener/slack/SlackBot.java | 28 ++++++-- .../slack/SlackNotificationTemplate.java | 45 +++++++++++-- .../slack/SlackNotificationTemplates.java | 5 +- .../slack/TestAbsentOrPattern.java | 65 +++++++++++++++++++ .../test/resources/slackNotifications.json | 14 ++++ 6 files changed, 203 insertions(+), 16 deletions(-) create mode 100644 twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/AbsentOrPattern.java create mode 100644 twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/slack/TestAbsentOrPattern.java diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/AbsentOrPattern.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/AbsentOrPattern.java new file mode 100644 index 000000000000..ebea93c4dcb6 --- /dev/null +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/AbsentOrPattern.java @@ -0,0 +1,62 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.twitter.presto.plugin.eventlistener.slack; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Optional; +import java.util.regex.Pattern; + +/** + * In addition to match a present string, this class defines a pattern to match absent string. + * Example: + * We have `principal` instantiated from its json definition: + * + * "principal": { + * "isAbsent": true, + * "regex": "user.*" + * } + * + * The `matches(Optional value)` will return `true` when either of conditions matches. + * + * principal.matches(Optional.empty()) -> true + * principal.matches(Optional.of("")) -> false + * principal.matches(Optional.of("user")) -> true + * principal.matches(Optional.of("user@example.top")) -> true + * + * The effective default value for "isAbsent" is `true`. + * The effective default value for "regex" is `.*`. + * + */ +public class AbsentOrPattern +{ + private final Optional isAbsent; + private final Optional regex; + + @JsonCreator + public AbsentOrPattern( + @JsonProperty("isAbsent") Optional isAbsent, + @JsonProperty("regex") Optional regex) + { + this.isAbsent = isAbsent; + this.regex = regex; + } + + public boolean matches(Optional value) + { + return value.map(v -> regex.map(r -> r.matcher(v).matches()).orElse(true)) + .orElse(isAbsent.orElse(true)); + } +} diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBot.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBot.java index 84bef01a4c93..a77e2ead3c81 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBot.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBot.java @@ -13,6 +13,7 @@ */ package com.twitter.presto.plugin.eventlistener.slack; +import com.facebook.presto.spi.ErrorType; import com.facebook.presto.spi.eventlistener.QueryCompletedEvent; import com.facebook.presto.spi.eventlistener.QueryCreatedEvent; import com.fasterxml.jackson.databind.DeserializationFeature; @@ -23,6 +24,7 @@ import com.twitter.presto.plugin.eventlistener.knowledge.KnowledgeBases; import io.airlift.json.ObjectMapperProvider; import io.airlift.log.Logger; +import io.airlift.units.Duration; import okhttp3.Authenticator; import okhttp3.Call; import okhttp3.Callback; @@ -43,6 +45,8 @@ import java.net.URI; import java.nio.file.Files; import java.nio.file.Paths; +import java.util.HashMap; +import java.util.Map; import java.util.Optional; import java.util.function.Consumer; import java.util.regex.Pattern; @@ -61,10 +65,10 @@ public class SlackBot private static final MediaType JSON_CONTENT_TYPE = MediaType.parse("Content-type: application/json; charset=utf-8"); private static final String USER = "\\$\\{USER}"; private static final String QUERY_ID = "\\$\\{QUERY_ID}"; - private static final String PRINCIPAL = "\\$\\{PRINCIPAL}"; private static final String STATE = "\\$\\{STATE}"; - private static final String FAILURE_MESSAGE = "\\$\\{FAILURE_MESSAGE}"; - private static final String FAILURE_TREATMENT = "\\$\\{FAILURE_TREATMENT}"; + public static final String PRINCIPAL = "\\$\\{PRINCIPAL}"; + public static final String FAILURE_MESSAGE = "\\$\\{FAILURE_MESSAGE}"; + public static final String FAILURE_TREATMENT = "\\$\\{FAILURE_TREATMENT}"; private static final String DASH = "-"; private static final String CREATED = "created"; private static final String COMPLETED = "completed"; @@ -120,6 +124,8 @@ public void handleQueryCreated(QueryCreatedEvent queryCreatedEvent) queryCreatedEvent.getMetadata().getQueryId(), queryCreatedEvent.getContext().getPrincipal(), queryCreatedEvent.getMetadata().getQueryState(), + Optional.empty(), + Optional.empty(), Optional.empty()); } @@ -131,19 +137,27 @@ public void handleQueryCompleted(QueryCompletedEvent queryCompletedEvent) queryCompletedEvent.getMetadata().getQueryId(), queryCompletedEvent.getContext().getPrincipal(), queryCompletedEvent.getMetadata().getQueryState(), - queryCompletedEvent.getFailureInfo().map(queryFailureInfo -> queryFailureInfo.getFailureMessage().orElse("unknown"))); + queryCompletedEvent.getFailureInfo().map(queryFailureInfo -> queryFailureInfo.getFailureMessage().orElse("unknown")), + Optional.of(Duration.succinctNanos(queryCompletedEvent.getStatistics().getWallTime().toNanos())), + queryCompletedEvent.getFailureInfo().map(queryFailureInfo -> queryFailureInfo.getErrorCode().getType())); } - private void handleSlackNotification(String event, String user, String queryId, Optional principal, String state, Optional failureMessage) + private void handleSlackNotification(String event, String user, String queryId, Optional principal, String state, Optional failureMessage, Optional wallTime, Optional errorType) { if (!slackUsers.matcher(user).matches()) { return; } - Optional template = notificationTemplates.getText(user, principal.orElse(DASH), event, state); + Optional treatment = failureMessage.map(message -> knowledgeBases.map(knowledge -> knowledge.getTreatment(message).orElse(DASH)).orElse(DASH)); + Map> fields = new HashMap<>(); + fields.put("principal", principal); + fields.put("failure_message", failureMessage); + fields.put("failure_treatment", treatment); + fields.put("wall_time", wallTime.map(Duration::toString)); + fields.put("error_type", errorType.map(ErrorType::toString)); + Optional template = notificationTemplates.getText(user, event, state, fields); if (!template.isPresent()) { return; } - Optional treatment = failureMessage.map(message -> knowledgeBases.map(knowledge -> knowledge.getTreatment(message).orElse(DASH)).orElse(DASH)); try { String email = emailTemplate.replaceAll(USER, user); String text = template.get() diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplate.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplate.java index f375f61f617a..0b65a4557b37 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplate.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplate.java @@ -15,41 +15,72 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import io.airlift.units.Duration; +import java.util.Map; import java.util.Optional; import java.util.regex.Pattern; public class SlackNotificationTemplate { + private static final Pattern ANY = Pattern.compile(".*"); private final String text; private final Optional userRegex; - private final Optional principalRegex; private final Optional eventRegex; private final Optional stateRegex; + private final Optional principal; + private final Optional minWallTime; + private final Optional errorTypeRegex; @JsonCreator public SlackNotificationTemplate( @JsonProperty("text") String text, @JsonProperty("user") Optional userRegex, - @JsonProperty("principal") Optional principalRegex, @JsonProperty("event") Optional eventRegex, - @JsonProperty("state") Optional stateRegex) + @JsonProperty("state") Optional stateRegex, + @JsonProperty("principal") Optional principal, + @JsonProperty("minWallTime") Optional minWallTime, + @JsonProperty("errorType") Optional errorTypeRegex) { this.text = text; this.userRegex = userRegex; - this.principalRegex = principalRegex; this.eventRegex = eventRegex; this.stateRegex = stateRegex; + this.principal = principal; + this.minWallTime = minWallTime; + this.errorTypeRegex = errorTypeRegex; } - public Optional match(String user, String principal, String event, String state) + public Optional match(String user, String event, String state, Map> fields) { if (userRegex.map(regex -> regex.matcher(user).matches()).orElse(true) && - principalRegex.map(regex -> regex.matcher(principal).matches()).orElse(true) && eventRegex.map(regex -> regex.matcher(event).matches()).orElse(true) && - stateRegex.map(regex -> regex.matcher(state).matches()).orElse(true)) { + stateRegex.map(regex -> regex.matcher(state).matches()).orElse(true) && + principal.map(pattern -> checkOptionalFields(pattern, "principal", fields)).orElse(true) && + errorTypeRegex.map(regex -> checkOptionalFields(regex, "error_type", fields)).orElse(true) && + minWallTime.map(regex -> checkOptionalMinimumDuration(regex, "wall_time", fields)).orElse(true) && + (!text.contains(SlackBot.FAILURE_MESSAGE) || checkOptionalFields(ANY, "failure_message", fields)) && + (!text.contains(SlackBot.FAILURE_TREATMENT) || checkOptionalFields(ANY, "failure_treatment", fields))) { return Optional.of(text); } return Optional.empty(); } + + private boolean checkOptionalFields(AbsentOrPattern pattern, String key, Map> fields) + { + Optional field = fields.getOrDefault(key, Optional.empty()); + return pattern.matches(field); + } + + private boolean checkOptionalFields(Pattern regex, String key, Map> fields) + { + Optional field = fields.getOrDefault(key, Optional.empty()); + return field.isPresent() && regex.matcher(field.get()).matches(); + } + + private boolean checkOptionalMinimumDuration(Duration minDuration, String key, Map> fields) + { + Optional field = fields.getOrDefault(key, Optional.empty()); + return field.isPresent() && minDuration.compareTo(Duration.valueOf(field.get())) <= 0; + } } diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplates.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplates.java index fce4caaaf4be..1ae722ebb67c 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplates.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackNotificationTemplates.java @@ -17,6 +17,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; +import java.util.Map; import java.util.Optional; public class SlackNotificationTemplates @@ -36,10 +37,10 @@ public List getTemplates() return templates; } - public Optional getText(String user, String principal, String event, String state) + public Optional getText(String user, String event, String state, Map> fields) { return templates.stream() - .map(template -> template.match(user, principal, event, state)) + .map(template -> template.match(user, event, state, fields)) .filter(Optional::isPresent) .map(Optional::get) .findFirst(); diff --git a/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/slack/TestAbsentOrPattern.java b/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/slack/TestAbsentOrPattern.java new file mode 100644 index 000000000000..9fbb2f413baa --- /dev/null +++ b/twitter-eventlistener-plugin/src/test/java/com/twitter/presto/plugin/eventlistener/slack/TestAbsentOrPattern.java @@ -0,0 +1,65 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.twitter.presto.plugin.eventlistener.slack; + +import org.testng.annotations.Test; + +import java.util.Optional; +import java.util.regex.Pattern; + +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; + +public class TestAbsentOrPattern +{ + @Test + public void testDefaults() + { + AbsentOrPattern pattern = new AbsentOrPattern(Optional.empty(), Optional.empty()); + assertTrue(pattern.matches(Optional.empty())); + assertTrue(pattern.matches(Optional.of(""))); + assertTrue(pattern.matches(Optional.of("user"))); + assertTrue(pattern.matches(Optional.of("user@exmaple.top"))); + } + + @Test + public void testExplicitMatchAbsentOnly() + { + AbsentOrPattern pattern = new AbsentOrPattern(Optional.empty(), Optional.of(Pattern.compile("a^"))); + assertTrue(pattern.matches(Optional.empty())); + assertFalse(pattern.matches(Optional.of(""))); + assertFalse(pattern.matches(Optional.of("user"))); + assertFalse(pattern.matches(Optional.of("user@exmaple.top"))); + } + + @Test + public void testExplicitMatchRegexOnly() + { + AbsentOrPattern pattern = new AbsentOrPattern(Optional.of(false), Optional.of(Pattern.compile("user.*"))); + assertFalse(pattern.matches(Optional.empty())); + assertFalse(pattern.matches(Optional.of(""))); + assertTrue(pattern.matches(Optional.of("user"))); + assertTrue(pattern.matches(Optional.of("user@exmaple.top"))); + } + + @Test + public void testExplicitMatchEither() + { + AbsentOrPattern pattern = new AbsentOrPattern(Optional.of(true), Optional.of(Pattern.compile("user.*"))); + assertTrue(pattern.matches(Optional.empty())); + assertFalse(pattern.matches(Optional.of(""))); + assertTrue(pattern.matches(Optional.of("user"))); + assertTrue(pattern.matches(Optional.of("user@exmaple.top"))); + } +} diff --git a/twitter-eventlistener-plugin/src/test/resources/slackNotifications.json b/twitter-eventlistener-plugin/src/test/resources/slackNotifications.json index cb4a7e077931..3128d4ac42b8 100644 --- a/twitter-eventlistener-plugin/src/test/resources/slackNotifications.json +++ b/twitter-eventlistener-plugin/src/test/resources/slackNotifications.json @@ -7,11 +7,25 @@ { "event": "completed", "state": "FINISHED", + "minWallTime": "1h", + "text": "The wall time should never reach 1 hour. So we should never see this message in tests." + }, + { + "event": "completed", + "state": "FINISHED", + "minWallTime": "1ms", "text": "I just completed your query." }, { "event": "completed", "state": "FAILED", + "errorType": "^(?!USER_ERROR).*$", + "text": "This should never reach because we only use USER_ERROR in tests." + }, + { + "event": "completed", + "state": "FAILED", + "errorType": "USER_ERROR", "text": "Unfortunately, your query was failed due to error: ${FAILURE_MESSAGE}" } ] From a9070f711bfbb5a785c61664c2eb1286982f7370 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 9 Jan 2019 13:18:08 -0800 Subject: [PATCH 320/331] Update version number to 0.210-tw-0.58 (#189) --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial-toolkit/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kudu/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory-context/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-password-authenticators/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-proxy/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 2 +- twitter-http-client/pom.xml | 2 +- 60 files changed, 61 insertions(+), 61 deletions(-) diff --git a/pom.xml b/pom.xml index 395564f2f078..d5ffddf9565a 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.210-tw-0.57 + 0.210-tw-0.58 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 6bda6e40582b..65d1ce4e1725 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index b6b2c397096b..7d81c71f8a94 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index aa1069d95319..9c129334a592 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index bcd5e534a109..7368a501b2eb 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index d30c40cfe9dd..0d7c25877cd1 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index f116fa5ffedf..f8b61501daa9 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.57 + 0.210-tw-0.58 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index de7606a1703e..7820f43980bb 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index e17cdafe697a..5c79c0e6e1d8 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index fdadad83589f..b115686cb604 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 5c30da225c79..2b78b9566999 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 06796a76eb4d..e7b187f35b6e 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 10313397d868..e485e7b324f4 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 5ddd4b98e087..257888ef4c68 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index c47f1acbf801..e8885e378266 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index a4040b320f51..4601d19ce54b 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index ce59496c4c69..05f2ac7468be 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index 5a4a9a839c75..dc356b5adb35 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index e8829e08f788..1dfac55cf2d0 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 567a1cb01470..d665e399f616 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index 1ff3f9e72643..a9e573e693a3 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-kafka diff --git a/presto-kudu/pom.xml b/presto-kudu/pom.xml index e3c617c37851..3f0be9ff3146 100644 --- a/presto-kudu/pom.xml +++ b/presto-kudu/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-kudu diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index fc649187ac0e..99b862d9cb99 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index c90677d349d8..b3e1d3d1bd7e 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index 5f6901a0d4fe..153139c257a0 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.210-tw-0.57 + 0.210-tw-0.58 presto-matching diff --git a/presto-memory-context/pom.xml b/presto-memory-context/pom.xml index 2a70dd3cd895..e2e20fd26177 100644 --- a/presto-memory-context/pom.xml +++ b/presto-memory-context/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-memory-context diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 373016839a32..f133657020d2 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 868eed70c4e6..de7b2cf3a7bb 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 8b444ccec8fd..d6129cf39e6c 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 2ff1c7ec8b50..38c068c5d71f 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index b2936c46b4a2..1984eb226525 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 18334bac206e..6999793225ca 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-parser diff --git a/presto-password-authenticators/pom.xml b/presto-password-authenticators/pom.xml index fe3f6b237ce1..001a48791990 100644 --- a/presto-password-authenticators/pom.xml +++ b/presto-password-authenticators/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-password-authenticators diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index ddebec1db5b5..b2d1ff8b637a 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index d43f0008fb15..4966f7805ea4 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 3f8d0921d3f0..d0d667829c79 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.57 + 0.210-tw-0.58 presto-product-tests diff --git a/presto-proxy/pom.xml b/presto-proxy/pom.xml index 22d3faca6dcb..2ec6d08f80eb 100644 --- a/presto-proxy/pom.xml +++ b/presto-proxy/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-proxy diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 90611d9081aa..ff2c4127662b 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index cec3cb4bdf3a..6b8c9da09f39 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index c4428c676d1a..7de987745eae 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 259afbebe561..7da459c7c602 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index 90ab5576f94b..9e5cbd474643 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 897acecbb519..a0ac1e6d5ae0 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index e28dd842a053..d77db20fecf5 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index a1f33205c5ca..53d7e02a185a 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 1042194da613..616af7d50071 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 8c07fc8a2231..4d44c5291873 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.210-tw-0.57 + 0.210-tw-0.58 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 36a41a60c174..2410a10b84df 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index d5b7e21210f8..4db78e961f25 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index f54af7df07ad..4a408748b230 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.57 + 0.210-tw-0.58 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index b295dc9c0cb9..956839c95099 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 33a21628b559..64eb54d2dea8 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index 0d1e6f29e317..51f13cd4aba3 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index c2eecfbd6b1b..e2fd41259311 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 6055b966abac..11f728979c93 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index 93644767f0c5..5c52f0110886 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-twitter-functions diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index cb070d5d6392..2973bd373aec 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index d64f4711fc95..760bcb542aed 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 33224d14b9a0..1784fbea24ff 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.57 + 0.210-tw-0.58 twitter-eventlistener-plugin diff --git a/twitter-http-client/pom.xml b/twitter-http-client/pom.xml index 02f499cb2f1f..0257083d2256 100644 --- a/twitter-http-client/pom.xml +++ b/twitter-http-client/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.57 + 0.210-tw-0.58 4.0.0 From ee90cf59101af555d0a750f4652002f88ccdf04c Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Tue, 15 Jan 2019 14:05:03 -0800 Subject: [PATCH 321/331] Correct the calculation of query wall time in query completion (#190) --- .../twitter/presto/plugin/eventlistener/slack/SlackBot.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBot.java b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBot.java index a77e2ead3c81..73aaaef50f10 100644 --- a/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBot.java +++ b/twitter-eventlistener-plugin/src/main/java/com/twitter/presto/plugin/eventlistener/slack/SlackBot.java @@ -48,6 +48,7 @@ import java.util.HashMap; import java.util.Map; import java.util.Optional; +import java.util.concurrent.TimeUnit; import java.util.function.Consumer; import java.util.regex.Pattern; @@ -138,7 +139,7 @@ public void handleQueryCompleted(QueryCompletedEvent queryCompletedEvent) queryCompletedEvent.getContext().getPrincipal(), queryCompletedEvent.getMetadata().getQueryState(), queryCompletedEvent.getFailureInfo().map(queryFailureInfo -> queryFailureInfo.getFailureMessage().orElse("unknown")), - Optional.of(Duration.succinctNanos(queryCompletedEvent.getStatistics().getWallTime().toNanos())), + Optional.of(Duration.succinctDuration(queryCompletedEvent.getEndTime().toEpochMilli() - queryCompletedEvent.getCreateTime().toEpochMilli(), TimeUnit.MILLISECONDS)), queryCompletedEvent.getFailureInfo().map(queryFailureInfo -> queryFailureInfo.getErrorCode().getType())); } From 9a083d34cab3e729e15df5c759afe33abdc98c97 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 17 Jan 2019 14:25:43 -0800 Subject: [PATCH 322/331] Update version number to 0.210-tw-0.59 (#191) --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial-toolkit/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kudu/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory-context/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-password-authenticators/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-proxy/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 2 +- twitter-http-client/pom.xml | 2 +- 60 files changed, 61 insertions(+), 61 deletions(-) diff --git a/pom.xml b/pom.xml index d5ffddf9565a..2f4a6f2f79ac 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.210-tw-0.58 + 0.210-tw-0.59 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 65d1ce4e1725..55be76924ff4 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 7d81c71f8a94..26212fc0156d 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 9c129334a592..53934b059126 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 7368a501b2eb..2e951cedfb9f 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 0d7c25877cd1..a77f86884786 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index f8b61501daa9..e73b467109d8 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.58 + 0.210-tw-0.59 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 7820f43980bb..363291ab1235 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 5c79c0e6e1d8..66fef8962c8d 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index b115686cb604..77f101742fa5 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 2b78b9566999..fab11e2edc60 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index e7b187f35b6e..fe807f64f417 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index e485e7b324f4..3fd7d6f18ca6 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index 257888ef4c68..a2135c107e1d 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index e8885e378266..3b67aaf0b837 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index 4601d19ce54b..ad4e91317571 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 05f2ac7468be..a3e47e819e69 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index dc356b5adb35..d52feb9ed029 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 1dfac55cf2d0..61a8ce88a3b4 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index d665e399f616..982f5c596895 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index a9e573e693a3..bd8e540ad668 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-kafka diff --git a/presto-kudu/pom.xml b/presto-kudu/pom.xml index 3f0be9ff3146..9bf6a8d3f2a6 100644 --- a/presto-kudu/pom.xml +++ b/presto-kudu/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-kudu diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 99b862d9cb99..2d351df013bc 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index b3e1d3d1bd7e..7f2de7754557 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index 153139c257a0..92de902054d9 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.210-tw-0.58 + 0.210-tw-0.59 presto-matching diff --git a/presto-memory-context/pom.xml b/presto-memory-context/pom.xml index e2e20fd26177..81d51f6464f5 100644 --- a/presto-memory-context/pom.xml +++ b/presto-memory-context/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-memory-context diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index f133657020d2..a7f9a568509d 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index de7b2cf3a7bb..96840a30d115 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index d6129cf39e6c..28177991630d 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 38c068c5d71f..f615cff7028b 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 1984eb226525..490e22b25117 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 6999793225ca..8af7fc8eec07 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-parser diff --git a/presto-password-authenticators/pom.xml b/presto-password-authenticators/pom.xml index 001a48791990..cb093b45e89c 100644 --- a/presto-password-authenticators/pom.xml +++ b/presto-password-authenticators/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-password-authenticators diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index b2d1ff8b637a..e98c8bf45f60 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 4966f7805ea4..689fb4265878 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index d0d667829c79..af5bd801e95b 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.58 + 0.210-tw-0.59 presto-product-tests diff --git a/presto-proxy/pom.xml b/presto-proxy/pom.xml index 2ec6d08f80eb..74f9b087fb36 100644 --- a/presto-proxy/pom.xml +++ b/presto-proxy/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-proxy diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index ff2c4127662b..e3f51ba06362 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 6b8c9da09f39..58a302d87a2f 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 7de987745eae..77acbcc252b8 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 7da459c7c602..657d9e4e26b0 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index 9e5cbd474643..4c79ba82c14f 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index a0ac1e6d5ae0..aff3a2cb6b3e 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index d77db20fecf5..614a98595bb7 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 53d7e02a185a..1c84b43fc725 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 616af7d50071..2624e0408d18 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 4d44c5291873..8c1d5896c7e0 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.210-tw-0.58 + 0.210-tw-0.59 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 2410a10b84df..277fd6b74e31 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 4db78e961f25..4a03e5b660eb 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 4a408748b230..e39ea369ef4f 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.58 + 0.210-tw-0.59 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index 956839c95099..26c50d7e4a35 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 64eb54d2dea8..43ec2ce46fa4 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index 51f13cd4aba3..d7ecb27b5114 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index e2fd41259311..3284b3de3afb 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index 11f728979c93..e9d2e95d9f5c 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index 5c52f0110886..7bcd8a5a63b5 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-twitter-functions diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index 2973bd373aec..79a5d816084b 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 760bcb542aed..2d5aad65778c 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 1784fbea24ff..b85e04ec5e46 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.58 + 0.210-tw-0.59 twitter-eventlistener-plugin diff --git a/twitter-http-client/pom.xml b/twitter-http-client/pom.xml index 0257083d2256..5f2f264ec1c1 100644 --- a/twitter-http-client/pom.xml +++ b/twitter-http-client/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.58 + 0.210-tw-0.59 4.0.0 From ea76111b80af8391a8e184f8b07cb5fc20bc3c9d Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Thu, 24 Jan 2019 18:49:19 -0800 Subject: [PATCH 323/331] Periodically re-materialize GSSCredential for KerberosAuthenticator (#192) Periodically re-materialize GSSCredential for KerberosAuthenticator and force session refresh for certain KerberosAuthenticator failure. --- .../security/KerberosAuthenticator.java | 5 +- .../server/security/ServerSecurityModule.java | 1 + .../security/KerberosAuthenticator.java | 279 ++++++++++++++++++ 3 files changed, 282 insertions(+), 3 deletions(-) create mode 100644 presto-main/src/main/java/com/twitter/presto/server/security/KerberosAuthenticator.java diff --git a/presto-main/src/main/java/com/facebook/presto/server/security/KerberosAuthenticator.java b/presto-main/src/main/java/com/facebook/presto/server/security/KerberosAuthenticator.java index 581b86ff6ac2..6ecd91a6f1dd 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/security/KerberosAuthenticator.java +++ b/presto-main/src/main/java/com/facebook/presto/server/security/KerberosAuthenticator.java @@ -64,9 +64,8 @@ public KerberosAuthenticator(KerberosConfig config) System.setProperty("java.security.krb5.conf", config.getKerberosConfig().getAbsolutePath()); try { - boolean isCompleteServicePrinciple = config.getServiceName().contains("@"); String hostname = InetAddress.getLocalHost().getCanonicalHostName().toLowerCase(Locale.US); - String servicePrincipal = isCompleteServicePrinciple ? config.getServiceName() : config.getServiceName() + "/" + hostname; + String servicePrincipal = config.getServiceName() + "/" + hostname; loginContext = new LoginContext("", null, null, new Configuration() { @Override @@ -92,7 +91,7 @@ public AppConfigurationEntry[] getAppConfigurationEntry(String name) loginContext.login(); serverCredential = doAs(loginContext.getSubject(), () -> gssManager.createCredential( - isCompleteServicePrinciple ? gssManager.createName(config.getServiceName(), GSSName.NT_USER_NAME) : gssManager.createName(config.getServiceName() + "@" + hostname, GSSName.NT_HOSTBASED_SERVICE), + gssManager.createName(config.getServiceName() + "@" + hostname, GSSName.NT_HOSTBASED_SERVICE), INDEFINITE_LIFETIME, new Oid[] { new Oid("1.2.840.113554.1.2.2"), // kerberos 5 diff --git a/presto-main/src/main/java/com/facebook/presto/server/security/ServerSecurityModule.java b/presto-main/src/main/java/com/facebook/presto/server/security/ServerSecurityModule.java index d65e0900a966..7926c6faba4b 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/security/ServerSecurityModule.java +++ b/presto-main/src/main/java/com/facebook/presto/server/security/ServerSecurityModule.java @@ -19,6 +19,7 @@ import com.google.inject.Provides; import com.google.inject.Scopes; import com.google.inject.multibindings.Multibinder; +import com.twitter.presto.server.security.KerberosAuthenticator; import io.airlift.configuration.AbstractConfigurationAwareModule; import io.airlift.http.server.TheServlet; diff --git a/presto-main/src/main/java/com/twitter/presto/server/security/KerberosAuthenticator.java b/presto-main/src/main/java/com/twitter/presto/server/security/KerberosAuthenticator.java new file mode 100644 index 000000000000..f739e2165293 --- /dev/null +++ b/presto-main/src/main/java/com/twitter/presto/server/security/KerberosAuthenticator.java @@ -0,0 +1,279 @@ +/* + * 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.twitter.presto.server.security; + +import com.facebook.presto.server.security.AuthenticationException; +import com.facebook.presto.server.security.Authenticator; +import com.facebook.presto.server.security.KerberosConfig; +import com.sun.security.auth.module.Krb5LoginModule; +import io.airlift.log.Logger; +import io.airlift.units.Duration; +import org.ietf.jgss.GSSContext; +import org.ietf.jgss.GSSCredential; +import org.ietf.jgss.GSSException; +import org.ietf.jgss.GSSManager; +import org.ietf.jgss.GSSName; +import org.ietf.jgss.Oid; +import org.joda.time.DateTime; + +import javax.annotation.PreDestroy; +import javax.inject.Inject; +import javax.security.auth.Subject; +import javax.security.auth.kerberos.KerberosPrincipal; +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 javax.servlet.http.HttpServletRequest; + +import java.io.File; +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.security.Principal; +import java.security.PrivilegedAction; +import java.util.Base64; +import java.util.HashMap; +import java.util.Locale; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.TimeUnit; + +import static com.google.common.net.HttpHeaders.AUTHORIZATION; +import static java.util.Objects.requireNonNull; +import static javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag.REQUIRED; +import static org.ietf.jgss.GSSCredential.ACCEPT_ONLY; +import static org.ietf.jgss.GSSCredential.INDEFINITE_LIFETIME; + +public class KerberosAuthenticator + implements Authenticator +{ + private static final Logger LOG = Logger.get(KerberosAuthenticator.class); + private static final Duration MIN_CREDENTIAL_LIFE_TIME = new Duration(60, TimeUnit.SECONDS); + private static final Duration DEFAULT_LIFT_TIME = new Duration(1, TimeUnit.HOURS); + + private static final String NEGOTIATE_SCHEME = "Negotiate"; + + private final GSSManager gssManager = GSSManager.getInstance(); + private final GSSName gssName; + private final File keyTab; + private final String servicePrincipal; + private Session serverSession; + + @Inject + public KerberosAuthenticator(KerberosConfig config) + { + System.setProperty("java.security.krb5.conf", config.getKerberosConfig().getAbsolutePath()); + + try { + boolean isCompleteServicePrinciple = config.getServiceName().contains("@"); + String hostname = InetAddress.getLocalHost().getCanonicalHostName().toLowerCase(Locale.US); + servicePrincipal = isCompleteServicePrinciple ? config.getServiceName() : config.getServiceName() + "/" + hostname; + gssName = isCompleteServicePrinciple ? gssManager.createName(config.getServiceName(), GSSName.NT_USER_NAME) : gssManager.createName(config.getServiceName() + "@" + hostname, GSSName.NT_HOSTBASED_SERVICE); + keyTab = config.getKeytab(); + } + catch (UnknownHostException | GSSException e) { + throw new RuntimeException(e); + } + } + + @PreDestroy + public void shutdown() + { + try { + getSession(false).getLoginContext().logout(); + } + catch (LoginException | GSSException e) { + throw new RuntimeException(e); + } + } + + @Override + public Principal authenticate(HttpServletRequest request) + throws AuthenticationException + { + String header = request.getHeader(AUTHORIZATION); + + String requestSpnegoToken = null; + + if (header != null) { + String[] parts = header.split("\\s+"); + if (parts.length == 2 && parts[0].equals(NEGOTIATE_SCHEME)) { + try { + requestSpnegoToken = parts[1]; + Optional principal = authenticate(parts[1]); + if (principal.isPresent()) { + return principal.get(); + } + } + catch (RuntimeException e) { + throw new RuntimeException("Authentication error for token: " + parts[1], e); + } + } + } + + if (requestSpnegoToken != null) { + throw new AuthenticationException("Authentication failed for token: " + requestSpnegoToken, NEGOTIATE_SCHEME); + } + + throw new AuthenticationException(null, NEGOTIATE_SCHEME); + } + + private Optional authenticate(String token) + { + try { + Session session = getSession(false); + LOG.debug("session remaining lift time: %d seconds", session.getRemainingLifetime()); + GSSContext context = doAs(session.getLoginContext().getSubject(), () -> gssManager.createContext(session.getServerCredential())); + + try { + byte[] inputToken = Base64.getDecoder().decode(token); + context.acceptSecContext(inputToken, 0, inputToken.length); + + // We can't hold on to the GSS context because HTTP is stateless, so fail + // if it can't be set up in a single challenge-response cycle + if (context.isEstablished()) { + return Optional.of(new KerberosPrincipal(context.getSrcName().toString())); + } + LOG.debug("Failed to establish GSS context for token %s", token); + } + catch (GSSException e) { + // ignore and fail the authentication + LOG.debug(e, "Authentication failed for token %s", token); + // try force session refresh for certain conditions + if (session.getAge() > MIN_CREDENTIAL_LIFE_TIME.getValue(TimeUnit.SECONDS) + && e.getMessage().contains("Cannot find key of appropriate type")) { + getSession(true); + } + } + finally { + try { + context.dispose(); + } + catch (GSSException e) { + // ignore + } + } + } + catch (LoginException | GSSException e) { + //ignore + LOG.debug(e, "Authenticator failed to get session"); + } + + return Optional.empty(); + } + + private synchronized Session getSession(boolean isForceRefresh) + throws LoginException, GSSException + { + if (isForceRefresh || serverSession == null || serverSession.getRemainingLifetime() < MIN_CREDENTIAL_LIFE_TIME.getValue(TimeUnit.SECONDS)) { + // TODO: do we need to call logout() on the LoginContext? + + LoginContext loginContext = new LoginContext("", null, null, new Configuration() + { + @Override + public AppConfigurationEntry[] getAppConfigurationEntry(String name) + { + Map options = new HashMap<>(); + options.put("refreshKrb5Config", "true"); + options.put("doNotPrompt", "true"); + if (LOG.isDebugEnabled()) { + options.put("debug", "true"); + } + if (keyTab != null) { + options.put("keyTab", keyTab.getAbsolutePath()); + } + options.put("isInitiator", "false"); + options.put("useKeyTab", "true"); + options.put("principal", servicePrincipal); + options.put("storeKey", "true"); + + return new AppConfigurationEntry[] {new AppConfigurationEntry(Krb5LoginModule.class.getName(), REQUIRED, options)}; + } + }); + loginContext.login(); + + GSSCredential serverCredential = doAs(loginContext.getSubject(), () -> gssManager.createCredential( + gssName, + INDEFINITE_LIFETIME, + new Oid[] { + new Oid("1.2.840.113554.1.2.2"), // kerberos 5 + new Oid("1.3.6.1.5.5.2") // spnego + }, + ACCEPT_ONLY)); + + serverSession = new Session(loginContext, serverCredential, (int) DEFAULT_LIFT_TIME.getValue(TimeUnit.SECONDS)); + } + + return serverSession; + } + + private interface GssSupplier + { + T get() + throws GSSException; + } + + private static T doAs(Subject subject, GssSupplier action) + { + return Subject.doAs(subject, (PrivilegedAction) () -> { + try { + return action.get(); + } + catch (GSSException e) { + throw new RuntimeException(e); + } + }); + } + + private static class Session + { + private final LoginContext loginContext; + private final GSSCredential serverCredential; + private final DateTime createdTime; + private final DateTime expiredTime; + + public Session(LoginContext loginContext, GSSCredential serverCredential, int lifetime) + { + requireNonNull(loginContext, "loginContext is null"); + requireNonNull(serverCredential, "gssCredential is null"); + + this.loginContext = loginContext; + this.serverCredential = serverCredential; + this.createdTime = DateTime.now(); + this.expiredTime = createdTime.plusSeconds(lifetime); + } + + public LoginContext getLoginContext() + { + return loginContext; + } + + public GSSCredential getServerCredential() + { + return serverCredential; + } + + public int getAge() + { + return (int) Duration.succinctDuration(DateTime.now().getMillis() - createdTime.getMillis(), TimeUnit.MILLISECONDS).getValue(TimeUnit.SECONDS); + } + + public int getRemainingLifetime() + throws GSSException + { + return Math.min(serverCredential.getRemainingLifetime(), + (int) Duration.succinctDuration(expiredTime.getMillis() - DateTime.now().getMillis(), TimeUnit.MILLISECONDS).getValue(TimeUnit.SECONDS)); + } + } +} From d54d324fb0e2e0a09887cea261ff8314b72f63a7 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Mon, 4 Feb 2019 11:23:52 -0800 Subject: [PATCH 324/331] Filter presto statement source when bypassing authentication (#193) --- .../server/security/AuthenticationFilter.java | 9 ++++++--- .../presto/server/security/SecurityConfig.java | 17 ++++++++--------- .../server/security/TestSecurityConfig.java | 8 ++++---- 3 files changed, 18 insertions(+), 16 deletions(-) diff --git a/presto-main/src/main/java/com/facebook/presto/server/security/AuthenticationFilter.java b/presto-main/src/main/java/com/facebook/presto/server/security/AuthenticationFilter.java index 9381da12d67c..cb0181f9b439 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/security/AuthenticationFilter.java +++ b/presto-main/src/main/java/com/facebook/presto/server/security/AuthenticationFilter.java @@ -34,6 +34,7 @@ import java.util.List; import java.util.Set; +import static com.facebook.presto.client.PrestoHeaders.PRESTO_SOURCE; import static com.google.common.io.ByteStreams.copy; import static com.google.common.io.ByteStreams.nullOutputStream; import static com.google.common.net.HttpHeaders.WWW_AUTHENTICATE; @@ -43,16 +44,17 @@ public class AuthenticationFilter implements Filter { + private static final String statementGetPathRegex = "\\/v1\\/statement\\/\\d{8}_\\d{6}_\\d{5}_\\w{5}\\/\\d+"; private final List authenticators; private final String httpAuthenticationPathRegex; - private final boolean allowByPass; + private final String statementSourceByPassRegex; @Inject public AuthenticationFilter(Set authenticators, SecurityConfig securityConfig) { this.authenticators = ImmutableList.copyOf(authenticators); this.httpAuthenticationPathRegex = requireNonNull(securityConfig.getHttpAuthenticationPathRegex(), "httpAuthenticationPathRegex is null"); - this.allowByPass = securityConfig.getAllowByPass(); + this.statementSourceByPassRegex = requireNonNull(securityConfig.getStatementSourceByPassRegex(), "statementSourceByPassRegex is null"); } @Override @@ -97,7 +99,8 @@ public void doFilter(ServletRequest servletRequest, ServletResponse servletRespo } // if authentication by pass allowed. - if (allowByPass) { + if ((request.getMethod().matches("GET") && request.getPathInfo().matches(statementGetPathRegex)) + || ((request.getHeader(PRESTO_SOURCE) != null && request.getHeader(PRESTO_SOURCE).matches(statementSourceByPassRegex)))) { nextFilter.doFilter(request, response); return; } diff --git a/presto-main/src/main/java/com/facebook/presto/server/security/SecurityConfig.java b/presto-main/src/main/java/com/facebook/presto/server/security/SecurityConfig.java index bf275b64a110..5c855d8abfa0 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/security/SecurityConfig.java +++ b/presto-main/src/main/java/com/facebook/presto/server/security/SecurityConfig.java @@ -33,9 +33,8 @@ public class SecurityConfig private List authenticationTypes = ImmutableList.of(); - private String httpAuthenticationPathRegex = "^\b$"; - - private boolean allowByPass; + private String httpAuthenticationPathRegex = "^\\b$"; + private String statementSourceByPassRegex = "^\\b$"; public enum AuthenticationType { @@ -86,16 +85,16 @@ public SecurityConfig setHttpAuthenticationPathRegex(String regex) return this; } - public boolean getAllowByPass() + public String getStatementSourceByPassRegex() { - return allowByPass; + return statementSourceByPassRegex; } - @Config("http-server.authentication.allow-by-pass") - @ConfigDescription("Allow authentication by pass") - public SecurityConfig setAllowByPass(boolean allowByPass) + @Config("http-server.statement.source.allow-by-pass-authentication") + @ConfigDescription("Regex of the statement source that allows bypass authentication") + public SecurityConfig setStatementSourceByPassRegex(String regex) { - this.allowByPass = allowByPass; + this.statementSourceByPassRegex = regex; return this; } } diff --git a/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java b/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java index 777360358857..a58772963eba 100644 --- a/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java +++ b/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java @@ -30,8 +30,8 @@ public void testDefaults() { ConfigAssertions.assertRecordedDefaults(ConfigAssertions.recordDefaults(SecurityConfig.class) .setAuthenticationTypes("") - .setHttpAuthenticationPathRegex("^\b$") - .setAllowByPass(false)); + .setHttpAuthenticationPathRegex("^\\b$") + .setStatementSourceByPassRegex("^\\b$")); } @Test @@ -40,13 +40,13 @@ public void testExplicitPropertyMappings() Map properties = new ImmutableMap.Builder() .put("http-server.authentication.type", "KERBEROS,PASSWORD") .put("http-server.http.authentication.path.regex", "^/v1/statement") - .put("http-server.authentication.allow-by-pass", "true") + .put("http-server.statement.source.allow-by-pass-authentication", "odbc|presto-jdbc") .build(); SecurityConfig expected = new SecurityConfig() .setAuthenticationTypes(ImmutableList.of(KERBEROS, PASSWORD)) .setHttpAuthenticationPathRegex("^/v1/statement") - .setAllowByPass(true); + .setStatementSourceByPassRegex("odbc|presto-jdbc"); ConfigAssertions.assertFullMapping(properties, expected); } From 73558cc7d871c6cf99e1bbccef85d887be65cfbd Mon Sep 17 00:00:00 2001 From: Hao Luo Date: Wed, 6 Feb 2019 18:50:56 -0800 Subject: [PATCH 325/331] Add dynamic batch sizing in Parquet reader (#194) * Add dynamic batch sizing in Parquet reader backport bf01fcdc5e79f2a15c71044372662bba04c35896 from prestosql/presto --- .../presto/hive/HiveClientConfig.java | 14 ++++ .../presto/hive/HiveSessionProperties.java | 11 +++ .../parquet/ParquetPageSourceFactory.java | 7 +- .../hive/parquet/reader/ParquetReader.java | 34 ++++++-- .../presto/hive/TestHiveClientConfig.java | 3 + .../parquet/AbstractTestParquetReader.java | 79 +++++++++++++++++++ .../presto/hive/parquet/ParquetTester.java | 64 +++++++++++++++ 7 files changed, 206 insertions(+), 6 deletions(-) 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 9b34f821c132..3c142b4405ce 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 @@ -99,6 +99,7 @@ public class HiveClientConfig private boolean useParquetColumnNames; private boolean parquetOptimizedReaderEnabled = true; private boolean parquetPredicatePushdownEnabled = true; + private DataSize parquetMaxReadBlockSize = new DataSize(16, MEGABYTE); private boolean assumeCanonicalPartitionKeys; @@ -693,6 +694,19 @@ public HiveClientConfig setParquetOptimizedReaderEnabled(boolean parquetOptimize return this; } + @NotNull + public DataSize getParquetMaxReadBlockSize() + { + return parquetMaxReadBlockSize; + } + + @Config("hive.parquet.max-read-block-size") + public HiveClientConfig setParquetMaxReadBlockSize(DataSize parquetMaxReadBlockSize) + { + this.parquetMaxReadBlockSize = parquetMaxReadBlockSize; + return this; + } + public boolean isUseOrcColumnNames() { return useOrcColumnNames; 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 62988901396a..acf3f68b4d2e 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 @@ -64,6 +64,7 @@ public final class HiveSessionProperties 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 PARQUET_USE_COLUMN_NAME = "parquet_use_column_names"; + private static final String PARQUET_MAX_READ_BLOCK_SIZE = "parquet_max_read_block_size"; private static final String MAX_SPLIT_SIZE = "max_split_size"; private static final String MAX_INITIAL_SPLIT_SIZE = "max_initial_split_size"; public static final String RCFILE_OPTIMIZED_WRITER_ENABLED = "rcfile_optimized_writer_enabled"; @@ -221,6 +222,11 @@ public HiveSessionProperties(HiveClientConfig hiveClientConfig, OrcFileWriterCon "Experimental: Parquet: Access Parquet columns using names from the file", hiveClientConfig.isUseParquetColumnNames(), false), + dataSizeSessionProperty( + PARQUET_MAX_READ_BLOCK_SIZE, + "Parquet: Maximum size of a block to read", + hiveClientConfig.getParquetMaxReadBlockSize(), + false), dataSizeSessionProperty( MAX_SPLIT_SIZE, "Max split size", @@ -398,6 +404,11 @@ public static boolean isUseParquetColumnNames(ConnectorSession session) return session.getProperty(PARQUET_USE_COLUMN_NAME, Boolean.class); } + public static DataSize getParquetMaxReadBlockSize(ConnectorSession session) + { + return session.getProperty(PARQUET_MAX_READ_BLOCK_SIZE, DataSize.class); + } + public static DataSize getMaxSplitSize(ConnectorSession session) { return session.getProperty(MAX_SPLIT_SIZE, DataSize.class); 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 5dd5a33f081b..28c80ffabc14 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 @@ -27,6 +27,7 @@ import com.facebook.presto.spi.predicate.TupleDomain; import com.facebook.presto.spi.type.TypeManager; import com.google.common.collect.ImmutableSet; +import io.airlift.units.DataSize; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; @@ -54,6 +55,7 @@ import static com.facebook.presto.hive.HiveColumnHandle.ColumnType.REGULAR; import static com.facebook.presto.hive.HiveErrorCode.HIVE_CANNOT_OPEN_SPLIT; import static com.facebook.presto.hive.HiveErrorCode.HIVE_MISSING_DATA; +import static com.facebook.presto.hive.HiveSessionProperties.getParquetMaxReadBlockSize; import static com.facebook.presto.hive.HiveSessionProperties.isParquetOptimizedReaderEnabled; import static com.facebook.presto.hive.HiveSessionProperties.isParquetPredicatePushdownEnabled; import static com.facebook.presto.hive.HiveSessionProperties.isUseParquetColumnNames; @@ -123,6 +125,7 @@ public Optional createPageSource( schema, columns, isUseParquetColumnNames(session), + getParquetMaxReadBlockSize(session), typeManager, isParquetPredicatePushdownEnabled(session), effectivePredicate, @@ -140,6 +143,7 @@ public static ParquetPageSource createParquetPageSource( Properties schema, List columns, boolean useParquetColumnNames, + DataSize maxReadBlockSize, TypeManager typeManager, boolean predicatePushdownEnabled, TupleDomain effectivePredicate, @@ -186,7 +190,8 @@ public static ParquetPageSource createParquetPageSource( messageColumnIO, blocks, dataSource, - systemMemoryContext); + systemMemoryContext, + maxReadBlockSize); return new ParquetPageSource( parquetReader, 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 cf6b544fadf2..ad907f69dafd 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 @@ -28,6 +28,7 @@ import com.facebook.presto.spi.type.MapType; import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.TypeSignatureParameter; +import io.airlift.units.DataSize; import it.unimi.dsi.fastutil.booleans.BooleanArrayList; import it.unimi.dsi.fastutil.booleans.BooleanList; import it.unimi.dsi.fastutil.ints.IntArrayList; @@ -51,6 +52,7 @@ import static com.facebook.presto.spi.type.StandardTypes.MAP; import static com.facebook.presto.spi.type.StandardTypes.ROW; import static com.google.common.base.Preconditions.checkArgument; +import static java.lang.Math.max; import static java.lang.Math.min; import static java.lang.Math.toIntExact; import static java.util.Objects.requireNonNull; @@ -59,6 +61,8 @@ public class ParquetReader implements Closeable { private static final int MAX_VECTOR_LENGTH = 1024; + private static final int INITIAL_BATCH_SIZE = 1; + private static final int BATCH_SIZE_GROWTH_FACTOR = 2; private final List blocks; private final List columns; @@ -71,21 +75,29 @@ public class ParquetReader private long currentGroupRowCount; private long nextRowInGroup; private int batchSize; + private int nextBatchSize = INITIAL_BATCH_SIZE; private final ParquetPrimitiveColumnReader[] columnReaders; + private long[] maxBytesPerCell; + private long maxCombinedBytesPerRow; + private final long maxReadBlockBytes; + private int maxBatchSize = MAX_VECTOR_LENGTH; private AggregatedMemoryContext currentRowGroupMemoryContext; public ParquetReader(MessageColumnIO messageColumnIO, List blocks, ParquetDataSource dataSource, - AggregatedMemoryContext systemMemoryContext) + AggregatedMemoryContext systemMemoryContext, + DataSize maxReadBlockSize) { this.blocks = blocks; this.dataSource = requireNonNull(dataSource, "dataSource is null"); this.systemMemoryContext = requireNonNull(systemMemoryContext, "systemMemoryContext is null"); this.currentRowGroupMemoryContext = systemMemoryContext.newAggregatedMemoryContext(); + this.maxReadBlockBytes = requireNonNull(maxReadBlockSize, "maxReadBlockSize is null").toBytes(); columns = messageColumnIO.getLeaves(); columnReaders = new ParquetPrimitiveColumnReader[columns.size()]; + maxBytesPerCell = new long[columns.size()]; } @Override @@ -107,8 +119,9 @@ public int nextBatch() return -1; } - batchSize = toIntExact(min(MAX_VECTOR_LENGTH, currentGroupRowCount - nextRowInGroup)); - + batchSize = toIntExact(min(nextBatchSize, maxBatchSize)); + nextBatchSize = min(batchSize * BATCH_SIZE_GROWTH_FACTOR, MAX_VECTOR_LENGTH); + batchSize = toIntExact(min(batchSize, currentGroupRowCount - nextRowInGroup)); nextRowInGroup += batchSize; currentPosition += batchSize; Arrays.stream(columnReaders) @@ -193,7 +206,8 @@ private ColumnChunk readPrimitive(PrimitiveField field) throws IOException { ColumnDescriptor columnDescriptor = field.getDescriptor(); - ParquetPrimitiveColumnReader columnReader = columnReaders[field.getId()]; + int fieldId = field.getId(); + ParquetPrimitiveColumnReader columnReader = columnReaders[fieldId]; if (columnReader.getPageReader() == null) { validateParquet(currentBlockMetadata.getRowCount() > 0, "Row group has 0 rows"); ColumnChunkMetaData metadata = getColumnChunkMetaData(columnDescriptor); @@ -205,7 +219,17 @@ private ColumnChunk readPrimitive(PrimitiveField field) ParquetColumnChunk columnChunk = new ParquetColumnChunk(descriptor, buffer, 0); columnReader.setPageReader(columnChunk.readAllPages()); } - return columnReader.readPrimitive(field); + ColumnChunk columnChunk = columnReader.readPrimitive(field); + + // update max size per primitive column chunk + long bytesPerCell = columnChunk.getBlock().getSizeInBytes() / batchSize; + if (maxBytesPerCell[fieldId] < bytesPerCell) { + // update batch size + maxCombinedBytesPerRow = maxCombinedBytesPerRow - maxBytesPerCell[fieldId] + bytesPerCell; + maxBatchSize = toIntExact(min(maxBatchSize, max(1, maxReadBlockBytes / maxCombinedBytesPerRow))); + maxBytesPerCell[fieldId] = bytesPerCell; + } + return columnChunk; } private byte[] allocateBlock(int length) 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 e1a8c6b420ff..89519364df51 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 @@ -79,6 +79,7 @@ public void testDefaults() .setMaxOpenSortFiles(50) .setWriteValidationThreads(16) .setUseParquetColumnNames(false) + .setParquetMaxReadBlockSize(new DataSize(16, Unit.MEGABYTE)) .setUseOrcColumnNames(false) .setParquetPredicatePushdownEnabled(true) .setParquetOptimizedReaderEnabled(true) @@ -155,6 +156,7 @@ public void testExplicitPropertyMappings() .put("hive.max-concurrent-file-renames", "100") .put("hive.assume-canonical-partition-keys", "true") .put("hive.parquet.use-column-names", "true") + .put("hive.parquet.max-read-block-size", "66kB") .put("hive.orc.use-column-names", "true") .put("hive.parquet-predicate-pushdown.enabled", "false") .put("hive.parquet-optimized-reader.enabled", "false") @@ -227,6 +229,7 @@ public void testExplicitPropertyMappings() .setDomainSocketPath("/foo") .setS3FileSystemType(S3FileSystemType.EMRFS) .setUseParquetColumnNames(true) + .setParquetMaxReadBlockSize(new DataSize(66, Unit.KILOBYTE)) .setUseOrcColumnNames(true) .setParquetPredicatePushdownEnabled(false) .setParquetOptimizedReaderEnabled(false) diff --git a/presto-hive/src/test/java/com/facebook/presto/hive/parquet/AbstractTestParquetReader.java b/presto-hive/src/test/java/com/facebook/presto/hive/parquet/AbstractTestParquetReader.java index 0a24e8191855..41611a320899 100644 --- a/presto-hive/src/test/java/com/facebook/presto/hive/parquet/AbstractTestParquetReader.java +++ b/presto-hive/src/test/java/com/facebook/presto/hive/parquet/AbstractTestParquetReader.java @@ -27,6 +27,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Range; import com.google.common.primitives.Shorts; +import io.airlift.units.DataSize; import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveDecimalObjectInspector; @@ -1492,6 +1493,43 @@ private void setParquetLogging() Logger.getLogger("parquet.hadoop.ColumnChunkPageWriteStore").setLevel(Level.WARNING); } + @Test + public void testStructMaxReadBytes() + throws Exception + { + DataSize maxReadBlockSize = new DataSize(1_000, DataSize.Unit.BYTE); + List structValues = createTestStructs( + Collections.nCopies(500, String.join("", Collections.nCopies(33, "test"))), + Collections.nCopies(500, String.join("", Collections.nCopies(1, "test")))); + List structFieldNames = asList("a", "b"); + Type structType = RowType.from(asList(field("a", VARCHAR), field("b", VARCHAR))); + + tester.testMaxReadBytes( + getStandardStructObjectInspector(structFieldNames, asList(javaStringObjectInspector, javaStringObjectInspector)), + structValues, + structValues, + structType, + maxReadBlockSize); + } + + @Test + public void testArrayMaxReadBytes() + throws Exception + { + DataSize maxReadBlockSize = new DataSize(1_000, DataSize.Unit.BYTE); + Iterable> values = createFixedTestArrays(limit(cycle(asList(1, null, 3, 5, null, null, null, 7, 11, null, 13, 17)), 30_000)); + tester.testMaxReadBytes(getStandardListObjectInspector(javaIntObjectInspector), values, values, new ArrayType(INTEGER), maxReadBlockSize); + } + + @Test + public void testMapMaxReadBytes() + throws Exception + { + DataSize maxReadBlockSize = new DataSize(1_000, DataSize.Unit.BYTE); + Iterable> values = createFixedTestMaps(Collections.nCopies(5_000, String.join("", Collections.nCopies(33, "test"))), longsBetween(0, 5_000)); + tester.testMaxReadBytes(getStandardMapObjectInspector(javaStringObjectInspector, javaLongObjectInspector), values, values, mapType(VARCHAR, BIGINT), maxReadBlockSize); + } + private static Iterable repeatEach(int n, Iterable iterable) { return () -> new AbstractIterator() @@ -1611,6 +1649,47 @@ private Iterable> createNullableTestArrays(Iterable values) return insertNullEvery(ThreadLocalRandom.current().nextInt(2, 5), createTestArrays(values)); } + private List> createFixedTestArrays(Iterable values) + { + List> arrays = new ArrayList<>(); + Iterator valuesIter = values.iterator(); + List array = new ArrayList<>(); + int count = 1; + while (valuesIter.hasNext()) { + if (count % 10 == 0) { + arrays.add(array); + array = new ArrayList<>(); + } + if (count % 20 == 0) { + arrays.add(Collections.emptyList()); + } + array.add(valuesIter.next()); + ++count; + } + return arrays; + } + + private Iterable> createFixedTestMaps(Iterable keys, Iterable values) + { + List> maps = new ArrayList<>(); + Iterator keysIterator = keys.iterator(); + Iterator valuesIterator = values.iterator(); + Map map = new HashMap<>(); + int count = 1; + while (keysIterator.hasNext() && valuesIterator.hasNext()) { + if (count % 5 == 0) { + maps.add(map); + map = new HashMap<>(); + } + if (count % 10 == 0) { + maps.add(Collections.emptyMap()); + } + map.put(keysIterator.next(), valuesIterator.next()); + ++count; + } + return maps; + } + private Iterable> createTestMaps(Iterable keys, Iterable values) { List> maps = new ArrayList<>(); 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 5ee8a3489c51..d74389d85610 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 @@ -81,6 +81,7 @@ import java.util.Set; import static com.facebook.presto.hive.AbstractTestHiveFileFormats.getFieldFromCursor; +import static com.facebook.presto.hive.HiveSessionProperties.getParquetMaxReadBlockSize; import static com.facebook.presto.hive.HiveTestUtils.createTestHdfsEnvironment; import static com.facebook.presto.hive.HiveUtil.isArrayType; import static com.facebook.presto.hive.HiveUtil.isMapType; @@ -288,6 +289,60 @@ void assertRoundTrip(List objectInspectors, } } + void testMaxReadBytes(ObjectInspector objectInspector, Iterable writeValues, Iterable readValues, Type type, DataSize maxReadBlockSize) + throws Exception + { + assertMaxReadBytes(singletonList(objectInspector), new Iterable[] {writeValues}, new Iterable[] { + readValues}, TEST_COLUMN, singletonList(type), Optional.empty(), maxReadBlockSize); + } + + void assertMaxReadBytes( + List objectInspectors, + Iterable[] writeValues, + Iterable[] readValues, + List columnNames, + List columnTypes, + Optional parquetSchema, + DataSize maxReadBlockSize) + throws Exception + { + WriterVersion version = PARQUET_1_0; + CompressionCodecName compressionCodecName = UNCOMPRESSED; + HiveClientConfig config = new HiveClientConfig().setHiveStorageFormat(HiveStorageFormat.PARQUET).setUseParquetColumnNames(false).setParquetMaxReadBlockSize(maxReadBlockSize); + ConnectorSession session = new TestingConnectorSession(new HiveSessionProperties(config, new OrcFileWriterConfig()).getSessionProperties()); + + try (TempFile tempFile = new TempFile("test", "parquet")) { + JobConf jobConf = new JobConf(); + jobConf.setEnum(COMPRESSION, compressionCodecName); + jobConf.setBoolean(ENABLE_DICTIONARY, true); + jobConf.setEnum(WRITER_VERSION, version); + writeParquetColumn( + jobConf, + tempFile.getFile(), + compressionCodecName, + createTableProperties(columnNames, objectInspectors), + getStandardStructObjectInspector(columnNames, objectInspectors), + getIterators(writeValues), + parquetSchema, + false); + + Iterator[] expectedValues = getIterators(readValues); + try (ConnectorPageSource pageSource = getFileFormat().createFileFormatReader( + session, + HDFS_ENVIRONMENT, + tempFile.getFile(), + columnNames, + columnTypes)) { + assertPageSource( + columnTypes, + expectedValues, + pageSource, + Optional.of(getParquetMaxReadBlockSize(session).toBytes())); + assertFalse(stream(expectedValues).allMatch(Iterator::hasNext)); + } + } + } + private static void assertFileContents( ConnectorSession session, File dataFile, @@ -313,9 +368,18 @@ private static void assertFileContents( } private static void assertPageSource(List types, Iterator[] valuesByField, ConnectorPageSource pageSource) + { + assertPageSource(types, valuesByField, pageSource, Optional.empty()); + } + + private static void assertPageSource(List types, Iterator[] valuesByField, ConnectorPageSource pageSource, Optional maxReadBlockSize) { Page page; while ((page = pageSource.getNextPage()) != null) { + if (maxReadBlockSize.isPresent()) { + assertTrue(page.getPositionCount() == 1 || page.getSizeInBytes() <= maxReadBlockSize.get()); + } + for (int field = 0; field < page.getChannelCount(); field++) { Block block = page.getBlock(field); for (int i = 0; i < block.getPositionCount(); i++) { From 527a20f94418eef3d5a0c330fefc195e29be8940 Mon Sep 17 00:00:00 2001 From: Beinan Date: Mon, 11 Feb 2019 17:38:12 -0800 Subject: [PATCH 326/331] Improve the error message of schema mismatch (#195) --- .../hive/thrift/ThriftHiveRecordCursor.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java index 163f5f6af4a5..c17de71972c2 100644 --- a/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java +++ b/presto-hive/src/main/java/com/facebook/presto/twitter/hive/thrift/ThriftHiveRecordCursor.java @@ -14,6 +14,7 @@ package com.facebook.presto.twitter.hive.thrift; import com.facebook.presto.hive.HiveColumnHandle; +import com.facebook.presto.hive.HiveErrorCode; import com.facebook.presto.hive.HiveType; import com.facebook.presto.spi.PrestoException; import com.facebook.presto.spi.RecordCursor; @@ -466,6 +467,22 @@ public boolean isNull(int fieldId) private void parseColumn(int column) { Type type = types[column]; + try { + parseColumn(column, type); + } + catch (ClassCastException cause) { + Object rawValue = rowData.getFieldValueForThriftId(thriftIds[column]); + String actualTypeName = rawValue.getClass().getTypeName(); + throw new PrestoException( + HiveErrorCode.HIVE_BAD_DATA, + String.format("Schema mismatched on column %d: expected type is %s, but actual type is %s, value is %s", + column, type.getDisplayName(), actualTypeName, rawValue), + cause); + } + } + + private void parseColumn(int column, Type type) + { if (BOOLEAN.equals(type)) { parseBooleanColumn(column); } From 30e1ce210992309884755121bec6595cf1e3c295 Mon Sep 17 00:00:00 2001 From: Yaliang Wang Date: Wed, 20 Feb 2019 17:23:49 -0800 Subject: [PATCH 327/331] Update version number to 0.210-tw-0.60 (#198) --- pom.xml | 4 ++-- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial-toolkit/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kudu/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory-context/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-password-authenticators/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-proxy/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 2 +- twitter-http-client/pom.xml | 2 +- 60 files changed, 61 insertions(+), 61 deletions(-) diff --git a/pom.xml b/pom.xml index 2f4a6f2f79ac..fb6df86d7f2e 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 pom presto-root @@ -30,7 +30,7 @@ scm:git:git://github.com/twitter-forks/presto.git https://github.com/twitter-forks/presto - 0.210-tw-0.59 + 0.210-tw-0.60 diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 55be76924ff4..78563ea31330 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 26212fc0156d..644419256476 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index 53934b059126..d6a3d6e1ea5e 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index 2e951cedfb9f..d1147c17a5c2 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index a77f86884786..30b414a9506a 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index e73b467109d8..c7d320110858 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.59 + 0.210-tw-0.60 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 363291ab1235..2e2f7c012d89 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index 66fef8962c8d..abfa6d651722 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 77f101742fa5..716f5a8812f6 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index fab11e2edc60..871beaff4f53 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index fe807f64f417..2961f01db2b4 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 3fd7d6f18ca6..862211481b3c 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index a2135c107e1d..e53d1f8896a1 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index 3b67aaf0b837..e9ca32c7af13 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index ad4e91317571..14c6a9d85af1 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index a3e47e819e69..6db381eefeda 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index d52feb9ed029..ea8f461214f2 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index 61a8ce88a3b4..f673d4cfd238 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 982f5c596895..6ae2e9590ddf 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index bd8e540ad668..c3ae92b3c642 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-kafka diff --git a/presto-kudu/pom.xml b/presto-kudu/pom.xml index 9bf6a8d3f2a6..e67deca80059 100644 --- a/presto-kudu/pom.xml +++ b/presto-kudu/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-kudu diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index 2d351df013bc..b4457ad4441e 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index 7f2de7754557..e11f98b4f97f 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index 92de902054d9..8a3798079c61 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.210-tw-0.59 + 0.210-tw-0.60 presto-matching diff --git a/presto-memory-context/pom.xml b/presto-memory-context/pom.xml index 81d51f6464f5..58048b4dbb61 100644 --- a/presto-memory-context/pom.xml +++ b/presto-memory-context/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-memory-context diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index a7f9a568509d..67907de66371 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 96840a30d115..93f2f3389a0f 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index 28177991630d..cef0917c65fa 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index f615cff7028b..7a060f920e51 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 490e22b25117..7c6502b01d59 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index 8af7fc8eec07..e12739fc9766 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-parser diff --git a/presto-password-authenticators/pom.xml b/presto-password-authenticators/pom.xml index cb093b45e89c..3cdfae280efd 100644 --- a/presto-password-authenticators/pom.xml +++ b/presto-password-authenticators/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-password-authenticators diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index e98c8bf45f60..b016aa734331 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 689fb4265878..76ec472f9be4 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index af5bd801e95b..4c3f57bf93b8 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.59 + 0.210-tw-0.60 presto-product-tests diff --git a/presto-proxy/pom.xml b/presto-proxy/pom.xml index 74f9b087fb36..9b854e387a69 100644 --- a/presto-proxy/pom.xml +++ b/presto-proxy/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-proxy diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index e3f51ba06362..09ed8c98e384 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 58a302d87a2f..89054fdd7b59 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 77acbcc252b8..27001a5b0440 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 657d9e4e26b0..43246519ee05 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index 4c79ba82c14f..95065b61ca07 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index aff3a2cb6b3e..2820906518a9 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index 614a98595bb7..a822820bbc6f 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 1c84b43fc725..0306b0c08e2c 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index 2624e0408d18..e757441197ef 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 8c1d5896c7e0..126212e1c029 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.210-tw-0.59 + 0.210-tw-0.60 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 277fd6b74e31..18be098d043d 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 4a03e5b660eb..0f49ceb478b3 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index e39ea369ef4f..8037617b3cf6 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.59 + 0.210-tw-0.60 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index 26c50d7e4a35..016c2924af7b 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 43ec2ce46fa4..6ed9a62454bd 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index d7ecb27b5114..171c13e193b7 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index 3284b3de3afb..1f0b89a7caa8 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index e9d2e95d9f5c..ac09f9cbf214 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index 7bcd8a5a63b5..bf0431b402ad 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-twitter-functions diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index 79a5d816084b..d092a69fb6cc 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 2d5aad65778c..609fccdde8d6 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index b85e04ec5e46..04c0ddeb72d0 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.59 + 0.210-tw-0.60 twitter-eventlistener-plugin diff --git a/twitter-http-client/pom.xml b/twitter-http-client/pom.xml index 5f2f264ec1c1..86753daa0f9f 100644 --- a/twitter-http-client/pom.xml +++ b/twitter-http-client/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.59 + 0.210-tw-0.60 4.0.0 From 4e4eef2c1ae721b6ab3d73e65f06bef6e33337ba Mon Sep 17 00:00:00 2001 From: Hao Luo Date: Thu, 23 May 2019 16:08:15 -0700 Subject: [PATCH 328/331] Revert "Filter presto statement source when bypassing authentication (#193)" This reverts commit d54d324fb0e2e0a09887cea261ff8314b72f63a7. --- .../server/security/AuthenticationFilter.java | 9 +++------ .../presto/server/security/SecurityConfig.java | 17 +++++++++-------- .../server/security/TestSecurityConfig.java | 8 ++++---- 3 files changed, 16 insertions(+), 18 deletions(-) diff --git a/presto-main/src/main/java/com/facebook/presto/server/security/AuthenticationFilter.java b/presto-main/src/main/java/com/facebook/presto/server/security/AuthenticationFilter.java index cb0181f9b439..9381da12d67c 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/security/AuthenticationFilter.java +++ b/presto-main/src/main/java/com/facebook/presto/server/security/AuthenticationFilter.java @@ -34,7 +34,6 @@ import java.util.List; import java.util.Set; -import static com.facebook.presto.client.PrestoHeaders.PRESTO_SOURCE; import static com.google.common.io.ByteStreams.copy; import static com.google.common.io.ByteStreams.nullOutputStream; import static com.google.common.net.HttpHeaders.WWW_AUTHENTICATE; @@ -44,17 +43,16 @@ public class AuthenticationFilter implements Filter { - private static final String statementGetPathRegex = "\\/v1\\/statement\\/\\d{8}_\\d{6}_\\d{5}_\\w{5}\\/\\d+"; private final List authenticators; private final String httpAuthenticationPathRegex; - private final String statementSourceByPassRegex; + private final boolean allowByPass; @Inject public AuthenticationFilter(Set authenticators, SecurityConfig securityConfig) { this.authenticators = ImmutableList.copyOf(authenticators); this.httpAuthenticationPathRegex = requireNonNull(securityConfig.getHttpAuthenticationPathRegex(), "httpAuthenticationPathRegex is null"); - this.statementSourceByPassRegex = requireNonNull(securityConfig.getStatementSourceByPassRegex(), "statementSourceByPassRegex is null"); + this.allowByPass = securityConfig.getAllowByPass(); } @Override @@ -99,8 +97,7 @@ public void doFilter(ServletRequest servletRequest, ServletResponse servletRespo } // if authentication by pass allowed. - if ((request.getMethod().matches("GET") && request.getPathInfo().matches(statementGetPathRegex)) - || ((request.getHeader(PRESTO_SOURCE) != null && request.getHeader(PRESTO_SOURCE).matches(statementSourceByPassRegex)))) { + if (allowByPass) { nextFilter.doFilter(request, response); return; } diff --git a/presto-main/src/main/java/com/facebook/presto/server/security/SecurityConfig.java b/presto-main/src/main/java/com/facebook/presto/server/security/SecurityConfig.java index 5c855d8abfa0..bf275b64a110 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/security/SecurityConfig.java +++ b/presto-main/src/main/java/com/facebook/presto/server/security/SecurityConfig.java @@ -33,8 +33,9 @@ public class SecurityConfig private List authenticationTypes = ImmutableList.of(); - private String httpAuthenticationPathRegex = "^\\b$"; - private String statementSourceByPassRegex = "^\\b$"; + private String httpAuthenticationPathRegex = "^\b$"; + + private boolean allowByPass; public enum AuthenticationType { @@ -85,16 +86,16 @@ public SecurityConfig setHttpAuthenticationPathRegex(String regex) return this; } - public String getStatementSourceByPassRegex() + public boolean getAllowByPass() { - return statementSourceByPassRegex; + return allowByPass; } - @Config("http-server.statement.source.allow-by-pass-authentication") - @ConfigDescription("Regex of the statement source that allows bypass authentication") - public SecurityConfig setStatementSourceByPassRegex(String regex) + @Config("http-server.authentication.allow-by-pass") + @ConfigDescription("Allow authentication by pass") + public SecurityConfig setAllowByPass(boolean allowByPass) { - this.statementSourceByPassRegex = regex; + this.allowByPass = allowByPass; return this; } } diff --git a/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java b/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java index a58772963eba..777360358857 100644 --- a/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java +++ b/presto-main/src/test/java/com/facebook/presto/server/security/TestSecurityConfig.java @@ -30,8 +30,8 @@ public void testDefaults() { ConfigAssertions.assertRecordedDefaults(ConfigAssertions.recordDefaults(SecurityConfig.class) .setAuthenticationTypes("") - .setHttpAuthenticationPathRegex("^\\b$") - .setStatementSourceByPassRegex("^\\b$")); + .setHttpAuthenticationPathRegex("^\b$") + .setAllowByPass(false)); } @Test @@ -40,13 +40,13 @@ public void testExplicitPropertyMappings() Map properties = new ImmutableMap.Builder() .put("http-server.authentication.type", "KERBEROS,PASSWORD") .put("http-server.http.authentication.path.regex", "^/v1/statement") - .put("http-server.statement.source.allow-by-pass-authentication", "odbc|presto-jdbc") + .put("http-server.authentication.allow-by-pass", "true") .build(); SecurityConfig expected = new SecurityConfig() .setAuthenticationTypes(ImmutableList.of(KERBEROS, PASSWORD)) .setHttpAuthenticationPathRegex("^/v1/statement") - .setStatementSourceByPassRegex("odbc|presto-jdbc"); + .setAllowByPass(true); ConfigAssertions.assertFullMapping(properties, expected); } From cc135fc35de19d7d5a3cc7d17f6a82ceda1681cb Mon Sep 17 00:00:00 2001 From: Hao Luo Date: Thu, 23 May 2019 16:08:20 -0700 Subject: [PATCH 329/331] Revert "Periodically re-materialize GSSCredential for KerberosAuthenticator (#192)" This reverts commit ea76111b80af8391a8e184f8b07cb5fc20bc3c9d. --- .../security/KerberosAuthenticator.java | 5 +- .../server/security/ServerSecurityModule.java | 1 - .../security/KerberosAuthenticator.java | 279 ------------------ 3 files changed, 3 insertions(+), 282 deletions(-) delete mode 100644 presto-main/src/main/java/com/twitter/presto/server/security/KerberosAuthenticator.java diff --git a/presto-main/src/main/java/com/facebook/presto/server/security/KerberosAuthenticator.java b/presto-main/src/main/java/com/facebook/presto/server/security/KerberosAuthenticator.java index 6ecd91a6f1dd..581b86ff6ac2 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/security/KerberosAuthenticator.java +++ b/presto-main/src/main/java/com/facebook/presto/server/security/KerberosAuthenticator.java @@ -64,8 +64,9 @@ public KerberosAuthenticator(KerberosConfig config) System.setProperty("java.security.krb5.conf", config.getKerberosConfig().getAbsolutePath()); try { + boolean isCompleteServicePrinciple = config.getServiceName().contains("@"); String hostname = InetAddress.getLocalHost().getCanonicalHostName().toLowerCase(Locale.US); - String servicePrincipal = config.getServiceName() + "/" + hostname; + String servicePrincipal = isCompleteServicePrinciple ? config.getServiceName() : config.getServiceName() + "/" + hostname; loginContext = new LoginContext("", null, null, new Configuration() { @Override @@ -91,7 +92,7 @@ public AppConfigurationEntry[] getAppConfigurationEntry(String name) loginContext.login(); serverCredential = doAs(loginContext.getSubject(), () -> gssManager.createCredential( - gssManager.createName(config.getServiceName() + "@" + hostname, GSSName.NT_HOSTBASED_SERVICE), + isCompleteServicePrinciple ? gssManager.createName(config.getServiceName(), GSSName.NT_USER_NAME) : gssManager.createName(config.getServiceName() + "@" + hostname, GSSName.NT_HOSTBASED_SERVICE), INDEFINITE_LIFETIME, new Oid[] { new Oid("1.2.840.113554.1.2.2"), // kerberos 5 diff --git a/presto-main/src/main/java/com/facebook/presto/server/security/ServerSecurityModule.java b/presto-main/src/main/java/com/facebook/presto/server/security/ServerSecurityModule.java index 7926c6faba4b..d65e0900a966 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/security/ServerSecurityModule.java +++ b/presto-main/src/main/java/com/facebook/presto/server/security/ServerSecurityModule.java @@ -19,7 +19,6 @@ import com.google.inject.Provides; import com.google.inject.Scopes; import com.google.inject.multibindings.Multibinder; -import com.twitter.presto.server.security.KerberosAuthenticator; import io.airlift.configuration.AbstractConfigurationAwareModule; import io.airlift.http.server.TheServlet; diff --git a/presto-main/src/main/java/com/twitter/presto/server/security/KerberosAuthenticator.java b/presto-main/src/main/java/com/twitter/presto/server/security/KerberosAuthenticator.java deleted file mode 100644 index f739e2165293..000000000000 --- a/presto-main/src/main/java/com/twitter/presto/server/security/KerberosAuthenticator.java +++ /dev/null @@ -1,279 +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.twitter.presto.server.security; - -import com.facebook.presto.server.security.AuthenticationException; -import com.facebook.presto.server.security.Authenticator; -import com.facebook.presto.server.security.KerberosConfig; -import com.sun.security.auth.module.Krb5LoginModule; -import io.airlift.log.Logger; -import io.airlift.units.Duration; -import org.ietf.jgss.GSSContext; -import org.ietf.jgss.GSSCredential; -import org.ietf.jgss.GSSException; -import org.ietf.jgss.GSSManager; -import org.ietf.jgss.GSSName; -import org.ietf.jgss.Oid; -import org.joda.time.DateTime; - -import javax.annotation.PreDestroy; -import javax.inject.Inject; -import javax.security.auth.Subject; -import javax.security.auth.kerberos.KerberosPrincipal; -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 javax.servlet.http.HttpServletRequest; - -import java.io.File; -import java.net.InetAddress; -import java.net.UnknownHostException; -import java.security.Principal; -import java.security.PrivilegedAction; -import java.util.Base64; -import java.util.HashMap; -import java.util.Locale; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.TimeUnit; - -import static com.google.common.net.HttpHeaders.AUTHORIZATION; -import static java.util.Objects.requireNonNull; -import static javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag.REQUIRED; -import static org.ietf.jgss.GSSCredential.ACCEPT_ONLY; -import static org.ietf.jgss.GSSCredential.INDEFINITE_LIFETIME; - -public class KerberosAuthenticator - implements Authenticator -{ - private static final Logger LOG = Logger.get(KerberosAuthenticator.class); - private static final Duration MIN_CREDENTIAL_LIFE_TIME = new Duration(60, TimeUnit.SECONDS); - private static final Duration DEFAULT_LIFT_TIME = new Duration(1, TimeUnit.HOURS); - - private static final String NEGOTIATE_SCHEME = "Negotiate"; - - private final GSSManager gssManager = GSSManager.getInstance(); - private final GSSName gssName; - private final File keyTab; - private final String servicePrincipal; - private Session serverSession; - - @Inject - public KerberosAuthenticator(KerberosConfig config) - { - System.setProperty("java.security.krb5.conf", config.getKerberosConfig().getAbsolutePath()); - - try { - boolean isCompleteServicePrinciple = config.getServiceName().contains("@"); - String hostname = InetAddress.getLocalHost().getCanonicalHostName().toLowerCase(Locale.US); - servicePrincipal = isCompleteServicePrinciple ? config.getServiceName() : config.getServiceName() + "/" + hostname; - gssName = isCompleteServicePrinciple ? gssManager.createName(config.getServiceName(), GSSName.NT_USER_NAME) : gssManager.createName(config.getServiceName() + "@" + hostname, GSSName.NT_HOSTBASED_SERVICE); - keyTab = config.getKeytab(); - } - catch (UnknownHostException | GSSException e) { - throw new RuntimeException(e); - } - } - - @PreDestroy - public void shutdown() - { - try { - getSession(false).getLoginContext().logout(); - } - catch (LoginException | GSSException e) { - throw new RuntimeException(e); - } - } - - @Override - public Principal authenticate(HttpServletRequest request) - throws AuthenticationException - { - String header = request.getHeader(AUTHORIZATION); - - String requestSpnegoToken = null; - - if (header != null) { - String[] parts = header.split("\\s+"); - if (parts.length == 2 && parts[0].equals(NEGOTIATE_SCHEME)) { - try { - requestSpnegoToken = parts[1]; - Optional principal = authenticate(parts[1]); - if (principal.isPresent()) { - return principal.get(); - } - } - catch (RuntimeException e) { - throw new RuntimeException("Authentication error for token: " + parts[1], e); - } - } - } - - if (requestSpnegoToken != null) { - throw new AuthenticationException("Authentication failed for token: " + requestSpnegoToken, NEGOTIATE_SCHEME); - } - - throw new AuthenticationException(null, NEGOTIATE_SCHEME); - } - - private Optional authenticate(String token) - { - try { - Session session = getSession(false); - LOG.debug("session remaining lift time: %d seconds", session.getRemainingLifetime()); - GSSContext context = doAs(session.getLoginContext().getSubject(), () -> gssManager.createContext(session.getServerCredential())); - - try { - byte[] inputToken = Base64.getDecoder().decode(token); - context.acceptSecContext(inputToken, 0, inputToken.length); - - // We can't hold on to the GSS context because HTTP is stateless, so fail - // if it can't be set up in a single challenge-response cycle - if (context.isEstablished()) { - return Optional.of(new KerberosPrincipal(context.getSrcName().toString())); - } - LOG.debug("Failed to establish GSS context for token %s", token); - } - catch (GSSException e) { - // ignore and fail the authentication - LOG.debug(e, "Authentication failed for token %s", token); - // try force session refresh for certain conditions - if (session.getAge() > MIN_CREDENTIAL_LIFE_TIME.getValue(TimeUnit.SECONDS) - && e.getMessage().contains("Cannot find key of appropriate type")) { - getSession(true); - } - } - finally { - try { - context.dispose(); - } - catch (GSSException e) { - // ignore - } - } - } - catch (LoginException | GSSException e) { - //ignore - LOG.debug(e, "Authenticator failed to get session"); - } - - return Optional.empty(); - } - - private synchronized Session getSession(boolean isForceRefresh) - throws LoginException, GSSException - { - if (isForceRefresh || serverSession == null || serverSession.getRemainingLifetime() < MIN_CREDENTIAL_LIFE_TIME.getValue(TimeUnit.SECONDS)) { - // TODO: do we need to call logout() on the LoginContext? - - LoginContext loginContext = new LoginContext("", null, null, new Configuration() - { - @Override - public AppConfigurationEntry[] getAppConfigurationEntry(String name) - { - Map options = new HashMap<>(); - options.put("refreshKrb5Config", "true"); - options.put("doNotPrompt", "true"); - if (LOG.isDebugEnabled()) { - options.put("debug", "true"); - } - if (keyTab != null) { - options.put("keyTab", keyTab.getAbsolutePath()); - } - options.put("isInitiator", "false"); - options.put("useKeyTab", "true"); - options.put("principal", servicePrincipal); - options.put("storeKey", "true"); - - return new AppConfigurationEntry[] {new AppConfigurationEntry(Krb5LoginModule.class.getName(), REQUIRED, options)}; - } - }); - loginContext.login(); - - GSSCredential serverCredential = doAs(loginContext.getSubject(), () -> gssManager.createCredential( - gssName, - INDEFINITE_LIFETIME, - new Oid[] { - new Oid("1.2.840.113554.1.2.2"), // kerberos 5 - new Oid("1.3.6.1.5.5.2") // spnego - }, - ACCEPT_ONLY)); - - serverSession = new Session(loginContext, serverCredential, (int) DEFAULT_LIFT_TIME.getValue(TimeUnit.SECONDS)); - } - - return serverSession; - } - - private interface GssSupplier - { - T get() - throws GSSException; - } - - private static T doAs(Subject subject, GssSupplier action) - { - return Subject.doAs(subject, (PrivilegedAction) () -> { - try { - return action.get(); - } - catch (GSSException e) { - throw new RuntimeException(e); - } - }); - } - - private static class Session - { - private final LoginContext loginContext; - private final GSSCredential serverCredential; - private final DateTime createdTime; - private final DateTime expiredTime; - - public Session(LoginContext loginContext, GSSCredential serverCredential, int lifetime) - { - requireNonNull(loginContext, "loginContext is null"); - requireNonNull(serverCredential, "gssCredential is null"); - - this.loginContext = loginContext; - this.serverCredential = serverCredential; - this.createdTime = DateTime.now(); - this.expiredTime = createdTime.plusSeconds(lifetime); - } - - public LoginContext getLoginContext() - { - return loginContext; - } - - public GSSCredential getServerCredential() - { - return serverCredential; - } - - public int getAge() - { - return (int) Duration.succinctDuration(DateTime.now().getMillis() - createdTime.getMillis(), TimeUnit.MILLISECONDS).getValue(TimeUnit.SECONDS); - } - - public int getRemainingLifetime() - throws GSSException - { - return Math.min(serverCredential.getRemainingLifetime(), - (int) Duration.succinctDuration(expiredTime.getMillis() - DateTime.now().getMillis(), TimeUnit.MILLISECONDS).getValue(TimeUnit.SECONDS)); - } - } -} From b51d979a9abb0233ff49e9411d1c9cef8e64505d Mon Sep 17 00:00:00 2001 From: Hao Luo Date: Thu, 23 May 2019 16:11:37 -0700 Subject: [PATCH 330/331] Update version to 0.210-tw-0.61 --- pom.xml | 2 +- presto-accumulo/pom.xml | 2 +- presto-array/pom.xml | 2 +- presto-atop/pom.xml | 2 +- presto-base-jdbc/pom.xml | 2 +- presto-benchmark-driver/pom.xml | 2 +- presto-benchmark/pom.xml | 2 +- presto-benchto-benchmarks/pom.xml | 2 +- presto-blackhole/pom.xml | 2 +- presto-cassandra/pom.xml | 2 +- presto-cli/pom.xml | 2 +- presto-client/pom.xml | 2 +- presto-docs/pom.xml | 2 +- presto-example-http/pom.xml | 2 +- presto-geospatial-toolkit/pom.xml | 2 +- presto-geospatial/pom.xml | 2 +- presto-hive-hadoop2/pom.xml | 2 +- presto-hive/pom.xml | 2 +- presto-jdbc/pom.xml | 2 +- presto-jmx/pom.xml | 2 +- presto-kafka/pom.xml | 2 +- presto-kudu/pom.xml | 2 +- presto-local-file/pom.xml | 2 +- presto-main/pom.xml | 2 +- presto-matching/pom.xml | 2 +- presto-memory-context/pom.xml | 2 +- presto-memory/pom.xml | 2 +- presto-ml/pom.xml | 2 +- presto-mongodb/pom.xml | 2 +- presto-mysql/pom.xml | 2 +- presto-orc/pom.xml | 2 +- presto-parser/pom.xml | 2 +- presto-password-authenticators/pom.xml | 2 +- presto-plugin-toolkit/pom.xml | 2 +- presto-postgresql/pom.xml | 2 +- presto-product-tests/pom.xml | 2 +- presto-proxy/pom.xml | 2 +- presto-raptor/pom.xml | 2 +- presto-rcfile/pom.xml | 2 +- presto-record-decoder/pom.xml | 2 +- presto-redis/pom.xml | 2 +- presto-redshift/pom.xml | 2 +- presto-resource-group-managers/pom.xml | 2 +- presto-server-rpm/pom.xml | 2 +- presto-server/pom.xml | 2 +- presto-spi/pom.xml | 2 +- presto-sqlserver/pom.xml | 2 +- presto-teradata-functions/pom.xml | 2 +- presto-testing-server-launcher/pom.xml | 2 +- presto-tests/pom.xml | 2 +- presto-thrift-connector-api/pom.xml | 2 +- presto-thrift-connector/pom.xml | 2 +- presto-thrift-testing-server/pom.xml | 2 +- presto-tpcds/pom.xml | 2 +- presto-tpch/pom.xml | 2 +- presto-twitter-functions/pom.xml | 2 +- presto-twitter-server/pom.xml | 2 +- presto-verifier/pom.xml | 2 +- twitter-eventlistener-plugin/pom.xml | 2 +- twitter-http-client/pom.xml | 2 +- 60 files changed, 60 insertions(+), 60 deletions(-) diff --git a/pom.xml b/pom.xml index fb6df86d7f2e..69694888e2c5 100644 --- a/pom.xml +++ b/pom.xml @@ -10,7 +10,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 pom presto-root diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 78563ea31330..76cf92fd5c35 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-accumulo diff --git a/presto-array/pom.xml b/presto-array/pom.xml index 644419256476..471258613f92 100644 --- a/presto-array/pom.xml +++ b/presto-array/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-array diff --git a/presto-atop/pom.xml b/presto-atop/pom.xml index d6a3d6e1ea5e..48dc85b2ecd0 100644 --- a/presto-atop/pom.xml +++ b/presto-atop/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-atop diff --git a/presto-base-jdbc/pom.xml b/presto-base-jdbc/pom.xml index d1147c17a5c2..6565dcccf164 100644 --- a/presto-base-jdbc/pom.xml +++ b/presto-base-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-base-jdbc diff --git a/presto-benchmark-driver/pom.xml b/presto-benchmark-driver/pom.xml index 30b414a9506a..54c8a31f79fd 100644 --- a/presto-benchmark-driver/pom.xml +++ b/presto-benchmark-driver/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-benchmark-driver diff --git a/presto-benchmark/pom.xml b/presto-benchmark/pom.xml index c7d320110858..b4c868014a7e 100644 --- a/presto-benchmark/pom.xml +++ b/presto-benchmark/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.60 + 0.210-tw-0.61 presto-benchmark diff --git a/presto-benchto-benchmarks/pom.xml b/presto-benchto-benchmarks/pom.xml index 2e2f7c012d89..19150483eb89 100644 --- a/presto-benchto-benchmarks/pom.xml +++ b/presto-benchto-benchmarks/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-benchto-benchmarks diff --git a/presto-blackhole/pom.xml b/presto-blackhole/pom.xml index abfa6d651722..65ffdb740482 100644 --- a/presto-blackhole/pom.xml +++ b/presto-blackhole/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-blackhole diff --git a/presto-cassandra/pom.xml b/presto-cassandra/pom.xml index 716f5a8812f6..b49390f7cdb0 100644 --- a/presto-cassandra/pom.xml +++ b/presto-cassandra/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-cassandra diff --git a/presto-cli/pom.xml b/presto-cli/pom.xml index 871beaff4f53..6e52e9fd5898 100644 --- a/presto-cli/pom.xml +++ b/presto-cli/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-cli diff --git a/presto-client/pom.xml b/presto-client/pom.xml index 2961f01db2b4..dc59e0869afc 100644 --- a/presto-client/pom.xml +++ b/presto-client/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-client diff --git a/presto-docs/pom.xml b/presto-docs/pom.xml index 862211481b3c..5e08593f7e3b 100644 --- a/presto-docs/pom.xml +++ b/presto-docs/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-docs diff --git a/presto-example-http/pom.xml b/presto-example-http/pom.xml index e53d1f8896a1..dbe8ae9cbc8a 100644 --- a/presto-example-http/pom.xml +++ b/presto-example-http/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-example-http diff --git a/presto-geospatial-toolkit/pom.xml b/presto-geospatial-toolkit/pom.xml index e9ca32c7af13..b6322b80cd6c 100644 --- a/presto-geospatial-toolkit/pom.xml +++ b/presto-geospatial-toolkit/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-geospatial-toolkit diff --git a/presto-geospatial/pom.xml b/presto-geospatial/pom.xml index 14c6a9d85af1..70fdfddb1b52 100644 --- a/presto-geospatial/pom.xml +++ b/presto-geospatial/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-geospatial diff --git a/presto-hive-hadoop2/pom.xml b/presto-hive-hadoop2/pom.xml index 6db381eefeda..bd6c086edab3 100644 --- a/presto-hive-hadoop2/pom.xml +++ b/presto-hive-hadoop2/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-hive-hadoop2 diff --git a/presto-hive/pom.xml b/presto-hive/pom.xml index ea8f461214f2..48652ef46b99 100644 --- a/presto-hive/pom.xml +++ b/presto-hive/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-hive diff --git a/presto-jdbc/pom.xml b/presto-jdbc/pom.xml index f673d4cfd238..2e7cab64d071 100644 --- a/presto-jdbc/pom.xml +++ b/presto-jdbc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-jdbc diff --git a/presto-jmx/pom.xml b/presto-jmx/pom.xml index 6ae2e9590ddf..1d96d2fe2f0f 100644 --- a/presto-jmx/pom.xml +++ b/presto-jmx/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-jmx diff --git a/presto-kafka/pom.xml b/presto-kafka/pom.xml index c3ae92b3c642..850cf9e6d37f 100644 --- a/presto-kafka/pom.xml +++ b/presto-kafka/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-kafka diff --git a/presto-kudu/pom.xml b/presto-kudu/pom.xml index e67deca80059..224a8826bc6a 100644 --- a/presto-kudu/pom.xml +++ b/presto-kudu/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-kudu diff --git a/presto-local-file/pom.xml b/presto-local-file/pom.xml index b4457ad4441e..04199e8dd06a 100644 --- a/presto-local-file/pom.xml +++ b/presto-local-file/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-local-file diff --git a/presto-main/pom.xml b/presto-main/pom.xml index e11f98b4f97f..3edd1dfa0b7b 100644 --- a/presto-main/pom.xml +++ b/presto-main/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-main diff --git a/presto-matching/pom.xml b/presto-matching/pom.xml index 8a3798079c61..4751fad8b140 100644 --- a/presto-matching/pom.xml +++ b/presto-matching/pom.xml @@ -18,7 +18,7 @@ presto-root com.facebook.presto - 0.210-tw-0.60 + 0.210-tw-0.61 presto-matching diff --git a/presto-memory-context/pom.xml b/presto-memory-context/pom.xml index 58048b4dbb61..0b64e4201690 100644 --- a/presto-memory-context/pom.xml +++ b/presto-memory-context/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-memory-context diff --git a/presto-memory/pom.xml b/presto-memory/pom.xml index 67907de66371..f9d0af4aac28 100644 --- a/presto-memory/pom.xml +++ b/presto-memory/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-memory diff --git a/presto-ml/pom.xml b/presto-ml/pom.xml index 93f2f3389a0f..8fc8188257b2 100644 --- a/presto-ml/pom.xml +++ b/presto-ml/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-ml diff --git a/presto-mongodb/pom.xml b/presto-mongodb/pom.xml index cef0917c65fa..9481c4df5456 100644 --- a/presto-mongodb/pom.xml +++ b/presto-mongodb/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-mongodb diff --git a/presto-mysql/pom.xml b/presto-mysql/pom.xml index 7a060f920e51..e9cfc03141a8 100644 --- a/presto-mysql/pom.xml +++ b/presto-mysql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-mysql diff --git a/presto-orc/pom.xml b/presto-orc/pom.xml index 7c6502b01d59..7fd98bc6b1b0 100644 --- a/presto-orc/pom.xml +++ b/presto-orc/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-orc diff --git a/presto-parser/pom.xml b/presto-parser/pom.xml index e12739fc9766..22f9be4e0a6a 100644 --- a/presto-parser/pom.xml +++ b/presto-parser/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-parser diff --git a/presto-password-authenticators/pom.xml b/presto-password-authenticators/pom.xml index 3cdfae280efd..7b53e0e8763c 100644 --- a/presto-password-authenticators/pom.xml +++ b/presto-password-authenticators/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-password-authenticators diff --git a/presto-plugin-toolkit/pom.xml b/presto-plugin-toolkit/pom.xml index b016aa734331..0487b866f46d 100644 --- a/presto-plugin-toolkit/pom.xml +++ b/presto-plugin-toolkit/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-plugin-toolkit diff --git a/presto-postgresql/pom.xml b/presto-postgresql/pom.xml index 76ec472f9be4..84471fd5616a 100644 --- a/presto-postgresql/pom.xml +++ b/presto-postgresql/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-postgresql diff --git a/presto-product-tests/pom.xml b/presto-product-tests/pom.xml index 4c3f57bf93b8..f9845d00bef5 100644 --- a/presto-product-tests/pom.xml +++ b/presto-product-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.60 + 0.210-tw-0.61 presto-product-tests diff --git a/presto-proxy/pom.xml b/presto-proxy/pom.xml index 9b854e387a69..47b1569375a5 100644 --- a/presto-proxy/pom.xml +++ b/presto-proxy/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-proxy diff --git a/presto-raptor/pom.xml b/presto-raptor/pom.xml index 09ed8c98e384..244e74bf9c4a 100644 --- a/presto-raptor/pom.xml +++ b/presto-raptor/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-raptor diff --git a/presto-rcfile/pom.xml b/presto-rcfile/pom.xml index 89054fdd7b59..f6dd8d27e924 100644 --- a/presto-rcfile/pom.xml +++ b/presto-rcfile/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-rcfile diff --git a/presto-record-decoder/pom.xml b/presto-record-decoder/pom.xml index 27001a5b0440..9a5825fa3d62 100644 --- a/presto-record-decoder/pom.xml +++ b/presto-record-decoder/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-record-decoder diff --git a/presto-redis/pom.xml b/presto-redis/pom.xml index 43246519ee05..32aa0a8eb44c 100644 --- a/presto-redis/pom.xml +++ b/presto-redis/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-redis diff --git a/presto-redshift/pom.xml b/presto-redshift/pom.xml index 95065b61ca07..1a0ac3d17620 100644 --- a/presto-redshift/pom.xml +++ b/presto-redshift/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-redshift diff --git a/presto-resource-group-managers/pom.xml b/presto-resource-group-managers/pom.xml index 2820906518a9..57c74af03a53 100644 --- a/presto-resource-group-managers/pom.xml +++ b/presto-resource-group-managers/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-resource-group-managers diff --git a/presto-server-rpm/pom.xml b/presto-server-rpm/pom.xml index a822820bbc6f..6b5d08274d8f 100644 --- a/presto-server-rpm/pom.xml +++ b/presto-server-rpm/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-server-rpm diff --git a/presto-server/pom.xml b/presto-server/pom.xml index 0306b0c08e2c..10cc4202477d 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-server diff --git a/presto-spi/pom.xml b/presto-spi/pom.xml index e757441197ef..dff5358fa8f8 100644 --- a/presto-spi/pom.xml +++ b/presto-spi/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-spi diff --git a/presto-sqlserver/pom.xml b/presto-sqlserver/pom.xml index 126212e1c029..fbae25e22e26 100644 --- a/presto-sqlserver/pom.xml +++ b/presto-sqlserver/pom.xml @@ -3,7 +3,7 @@ presto-root com.facebook.presto - 0.210-tw-0.60 + 0.210-tw-0.61 4.0.0 diff --git a/presto-teradata-functions/pom.xml b/presto-teradata-functions/pom.xml index 18be098d043d..4eccbe025a8c 100644 --- a/presto-teradata-functions/pom.xml +++ b/presto-teradata-functions/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-teradata-functions diff --git a/presto-testing-server-launcher/pom.xml b/presto-testing-server-launcher/pom.xml index 0f49ceb478b3..cc16bbcc8c67 100644 --- a/presto-testing-server-launcher/pom.xml +++ b/presto-testing-server-launcher/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-testing-server-launcher diff --git a/presto-tests/pom.xml b/presto-tests/pom.xml index 8037617b3cf6..d2df45f3b222 100644 --- a/presto-tests/pom.xml +++ b/presto-tests/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.60 + 0.210-tw-0.61 presto-tests diff --git a/presto-thrift-connector-api/pom.xml b/presto-thrift-connector-api/pom.xml index 016c2924af7b..569aae519dd8 100644 --- a/presto-thrift-connector-api/pom.xml +++ b/presto-thrift-connector-api/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-thrift-connector-api diff --git a/presto-thrift-connector/pom.xml b/presto-thrift-connector/pom.xml index 6ed9a62454bd..d7768ec4bc0e 100644 --- a/presto-thrift-connector/pom.xml +++ b/presto-thrift-connector/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-thrift-connector diff --git a/presto-thrift-testing-server/pom.xml b/presto-thrift-testing-server/pom.xml index 171c13e193b7..318a09443bb0 100644 --- a/presto-thrift-testing-server/pom.xml +++ b/presto-thrift-testing-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-thrift-testing-server diff --git a/presto-tpcds/pom.xml b/presto-tpcds/pom.xml index 1f0b89a7caa8..70ba957695d6 100644 --- a/presto-tpcds/pom.xml +++ b/presto-tpcds/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-tpcds diff --git a/presto-tpch/pom.xml b/presto-tpch/pom.xml index ac09f9cbf214..080ee02d71b8 100644 --- a/presto-tpch/pom.xml +++ b/presto-tpch/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-tpch diff --git a/presto-twitter-functions/pom.xml b/presto-twitter-functions/pom.xml index bf0431b402ad..6a4be41af66a 100644 --- a/presto-twitter-functions/pom.xml +++ b/presto-twitter-functions/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-twitter-functions diff --git a/presto-twitter-server/pom.xml b/presto-twitter-server/pom.xml index d092a69fb6cc..6f81c054e26f 100644 --- a/presto-twitter-server/pom.xml +++ b/presto-twitter-server/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-twitter-server diff --git a/presto-verifier/pom.xml b/presto-verifier/pom.xml index 609fccdde8d6..49986d8f76aa 100644 --- a/presto-verifier/pom.xml +++ b/presto-verifier/pom.xml @@ -5,7 +5,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 presto-verifier diff --git a/twitter-eventlistener-plugin/pom.xml b/twitter-eventlistener-plugin/pom.xml index 04c0ddeb72d0..2b19dd8602a1 100644 --- a/twitter-eventlistener-plugin/pom.xml +++ b/twitter-eventlistener-plugin/pom.xml @@ -4,7 +4,7 @@ com.facebook.presto presto-root - 0.210-tw-0.60 + 0.210-tw-0.61 twitter-eventlistener-plugin diff --git a/twitter-http-client/pom.xml b/twitter-http-client/pom.xml index 86753daa0f9f..5ed9414a43b7 100644 --- a/twitter-http-client/pom.xml +++ b/twitter-http-client/pom.xml @@ -5,7 +5,7 @@ presto-root com.facebook.presto - 0.210-tw-0.60 + 0.210-tw-0.61 4.0.0 From 1ceb4a14867d4deac99ef79a2d6245adf95870dd Mon Sep 17 00:00:00 2001 From: Hao Luo Date: Mon, 19 Aug 2019 17:03:08 -0700 Subject: [PATCH 331/331] Fix VERSION_NUMBER pattern in JavaVersion --- .../src/main/java/com/facebook/presto/server/JavaVersion.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/presto-main/src/main/java/com/facebook/presto/server/JavaVersion.java b/presto-main/src/main/java/com/facebook/presto/server/JavaVersion.java index b5fd1ab4fafa..a7e72f2b6899 100644 --- a/presto-main/src/main/java/com/facebook/presto/server/JavaVersion.java +++ b/presto-main/src/main/java/com/facebook/presto/server/JavaVersion.java @@ -28,7 +28,7 @@ public class JavaVersion { // As described in JEP-223 - private static final String VERSION_NUMBER = "(?[1-9][0-9]*)(\\.(?(0|[1-9][0-9]*))(\\.(?:(0|[1-9][0-9]*)))?)?"; + private static final String VERSION_NUMBER = "(?[1-9][0-9]*)(\\.(?(0|[1-9][0-9]*))(\\.(?:(0|[1-9][0-9]*)))?)*"; private static final String PRE = "(?:-(?:[a-zA-Z0-9]+))?"; private static final String BUILD = "(?:(?:\\+)(?:0|[1-9][0-9]*)?)?"; private static final String OPT = "(?:-(?:[-a-zA-Z0-9.]+))?";

- $.ajax({url: 'v1/query/' + query.queryId + '/killed', type: 'PUT', data: "Killed via web UI"}) } className="btn btn-warning" target="_blank"> + $.ajax({url: '/v1/query/' + query.queryId + '/killed', type: 'PUT', data: "Killed via web UI"}) } className="btn btn-warning" target="_blank"> Kill