Skip to content

Commit b3a0ebd

Browse files
authored
Merge branch 'prestodb:master' into wanweitao/fix-presto-ui
2 parents e3a8ac8 + 5eaa476 commit b3a0ebd

File tree

68 files changed

+3736
-212
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

68 files changed

+3736
-212
lines changed

.github/workflows/prestocpp-macos-build.yml

Lines changed: 41 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -3,15 +3,33 @@ name: prestocpp-macos-build
33
on:
44
workflow_dispatch:
55
pull_request:
6-
paths:
7-
- 'presto-native-execution/**'
8-
- '.github/workflows/prestocpp-macos-build.yml'
6+
97
jobs:
8+
changes:
9+
runs-on: macos-15
10+
# Required permissions
11+
permissions:
12+
pull-requests: read
13+
# Set job outputs to values from filter step
14+
outputs:
15+
codechange: ${{ steps.filter.outputs.codechange }}
16+
steps:
17+
# For pull requests it's not necessary to checkout the code
18+
- name: Run changes check for PRs
19+
uses: dorny/paths-filter@7267a8516b6f92bdb098633497bad573efdbf271
20+
id: filter
21+
with:
22+
filters: |
23+
codechange:
24+
- 'presto-native-execution/**'
25+
- '.github/workflows/prestocpp-macos-build.yml'
26+
1027
prestocpp-macos-build-engine:
1128
strategy:
1229
matrix:
1330
os: [macos-13, macos-15]
1431
runs-on: ${{ matrix.os }}
32+
needs: changes
1533
env:
1634
CCACHE_DIR: "${{ github.workspace }}/ccache"
1735
CMAKE_POLICY_VERSION_MINIMUM: "3.5"
@@ -24,18 +42,26 @@ jobs:
2442
cancel-in-progress: true
2543
steps:
2644
- uses: actions/checkout@v4
45+
if: |
46+
needs.changes.outputs.codechange == 'true'
2747
2848
- name: Fix git permissions
49+
if: |
50+
needs.changes.outputs.codechange == 'true'
2951
# Usually actions/checkout does this but as we run in a container
3052
# it doesn't work
3153
run: git config --global --add safe.directory ${GITHUB_WORKSPACE}
3254

3355
- name: Update submodules
56+
if: |
57+
needs.changes.outputs.codechange == 'true'
3458
run: |
3559
cd presto-native-execution
3660
make submodules
3761
3862
- name: "Setup MacOS"
63+
if: |
64+
needs.changes.outputs.codechange == 'true'
3965
run: |
4066
set -xu
4167
source presto-native-execution/scripts/setup-macos.sh
@@ -68,18 +94,26 @@ jobs:
6894
brew link --force protobuf@21
6995
7096
- name: Install Github CLI for using apache/infrastructure-actions/stash
97+
if: |
98+
needs.changes.outputs.codechange == 'true'
7199
run: |
72100
brew install gh
73101
74102
- uses: apache/infrastructure-actions/stash/restore@4ab8682fbd4623d2b4fc1c98db38aba5091924c3
103+
if: |
104+
needs.changes.outputs.codechange == 'true'
75105
with:
76106
path: '${{ env.CCACHE_DIR }}'
77107
key: ccache-prestocpp-macos-build-engine-${{ matrix.os }}
78108

79109
- name: Zero ccache statistics
110+
if: |
111+
needs.changes.outputs.codechange == 'true'
80112
run: PATH=${INSTALL_PREFIX}/bin:${PATH} ccache -sz
81113

82114
- name: "Build presto_cpp on MacOS"
115+
if: |
116+
needs.changes.outputs.codechange == 'true'
83117
run: |
84118
clang --version
85119
export PATH=$(brew --prefix m4)/bin:$(brew --prefix bison)/bin:${PATH}
@@ -99,9 +133,13 @@ jobs:
99133
ninja -C _build/${BUILD_TYPE} -j ${NJOBS}
100134
101135
- name: Ccache after
136+
if: |
137+
needs.changes.outputs.codechange == 'true'
102138
run: PATH=${INSTALL_PREFIX}/bin:${PATH} ccache -s
103139

104140
- uses: apache/infrastructure-actions/stash/save@4ab8682fbd4623d2b4fc1c98db38aba5091924c3
141+
if: |
142+
needs.changes.outputs.codechange == 'true'
105143
with:
106144
path: '${{ env.CCACHE_DIR }}'
107145
key: ccache-prestocpp-macos-build-engine-${{ matrix.os }}

CONTRIBUTING.md

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -342,7 +342,7 @@ Note: Each PR/commit should have a single primary type. If your changes span mul
342342
* `spi` - Service Provider Interface changes
343343
* `scheduler` - Task scheduling and execution
344344
* `protocol` - Wire protocol and serialization
345-
* `connector` - Changes to broader connector functionality or conector SPI
345+
* `connector` - Changes to broader connector functionality or connector SPI
346346
* `resource` - Resource management (memory manager, resource groups)
347347
* `security` - Authentication and authorization
348348
* `function` - Built-in functions and operators

pom.xml

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -83,7 +83,7 @@
8383
<dep.commons.compress.version>1.26.2</dep.commons.compress.version>
8484
<dep.protobuf-java.version>4.29.0</dep.protobuf-java.version>
8585
<dep.jetty.version>12.0.18</dep.jetty.version>
86-
<dep.netty.version>4.1.126.Final</dep.netty.version>
86+
<dep.netty.version>4.1.128.Final</dep.netty.version>
8787
<dep.reactor-netty.version>1.2.8</dep.reactor-netty.version>
8888
<dep.snakeyaml.version>2.5</dep.snakeyaml.version>
8989
<dep.gson.version>2.12.1</dep.gson.version>

presto-docs/src/main/sphinx/connector/iceberg.rst

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -347,9 +347,9 @@ Property Name Description
347347

348348
``iceberg.compression-codec`` The compression codec to use when writing files. The ``GZIP`` Yes No, write is not supported yet
349349
available values are ``NONE``, ``SNAPPY``, ``GZIP``,
350-
and ``ZSTD``.
350+
``LZ4``, and ``ZSTD``.
351351

352-
Note: ``ZSTD`` is only available when
352+
Note: ``LZ4`` is only available when
353353
``iceberg.file-format=ORC``.
354354

355355

presto-docs/src/main/sphinx/functions/math.rst

Lines changed: 12 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -266,9 +266,14 @@ Probability Functions: cdf
266266
Compute the Poisson cdf with given lambda (mean) parameter: P(N <= value; lambda).
267267
The lambda parameter must be a positive real number (of type DOUBLE) and value must be a non-negative integer.
268268

269+
.. function:: t_cdf(df, value) -> double
270+
271+
Compute the Student's t cdf with given degrees of freedom: P(N < value; df).
272+
The degrees of freedom must be a positive real number and value must be a real value.
273+
269274
.. function:: weibull_cdf(a, b, value) -> double
270275

271-
Compute the Weibull cdf with given parameters a, b: P(N <= value). The ``a``
276+
Compute the Weibull cdf with given parameters a, b: P(N <= value). The ``a``
272277
and ``b`` parameters must be positive doubles and ``value`` must also be a double.
273278

274279

@@ -332,6 +337,12 @@ Probability Functions: inverse_cdf
332337
The lambda parameter must be a positive real number (of type DOUBLE).
333338
The probability p must lie on the interval [0, 1).
334339

340+
.. function:: inverse_t_cdf(df, p) -> double
341+
342+
Compute the inverse of the Student's t cdf with given degrees of freedom for the cumulative
343+
probability (p): P(N < n). The degrees of freedom must be a positive real value.
344+
The probability p must lie on the interval [0, 1].
345+
335346
.. function:: inverse_weibull_cdf(a, b, p) -> double
336347

337348
Compute the inverse of the Weibull cdf with given parameters ``a``, ``b`` for the probability ``p``.

presto-docs/src/main/sphinx/presto-cpp.rst

Lines changed: 8 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -84,4 +84,11 @@ For more information see :doc:`/connector/iceberg` documentation.
8484
TPCH Connector
8585
^^^^^^^^^^^^^^
8686

87-
* TPCH connector, with ``tpch.naming=standard`` catalog property.
87+
* TPCH connector, with ``tpch.naming=standard`` catalog property.
88+
89+
TPC-DS Connector
90+
^^^^^^^^^^^^^^^^
91+
92+
* TPC-DS connector, with ``tpcds.use-varchar-type=true`` in the coordinator's TPCDS catalog file.
93+
94+
For more information see :doc:`/connector/tpcds` documentation.

presto-hive/src/main/java/com/facebook/presto/hive/HiveCompressionCodec.java

Lines changed: 7 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,7 @@
1616
import com.facebook.presto.orc.metadata.CompressionKind;
1717
import org.apache.hadoop.io.compress.CompressionCodec;
1818
import org.apache.hadoop.io.compress.GzipCodec;
19+
import org.apache.hadoop.io.compress.Lz4Codec;
1920
import org.apache.hadoop.io.compress.SnappyCodec;
2021
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
2122

@@ -25,19 +26,20 @@
2526
import static com.facebook.presto.hive.HiveStorageFormat.DWRF;
2627
import static com.facebook.presto.hive.HiveStorageFormat.ORC;
2728
import static com.facebook.presto.hive.HiveStorageFormat.PAGEFILE;
29+
import static com.facebook.presto.hive.HiveStorageFormat.PARQUET;
2830
import static java.util.Objects.requireNonNull;
2931

3032
public enum HiveCompressionCodec
3133
{
3234
NONE(null, CompressionKind.NONE, CompressionCodecName.UNCOMPRESSED, f -> true),
3335
SNAPPY(SnappyCodec.class, CompressionKind.SNAPPY, CompressionCodecName.SNAPPY, f -> true),
3436
GZIP(GzipCodec.class, CompressionKind.ZLIB, CompressionCodecName.GZIP, f -> true),
35-
LZ4(null, CompressionKind.NONE, null, f -> f == PAGEFILE),
36-
ZSTD(null, CompressionKind.ZSTD, null, f -> f == ORC || f == DWRF || f == PAGEFILE);
37+
LZ4(Lz4Codec.class, CompressionKind.LZ4, CompressionCodecName.UNCOMPRESSED, f -> f == PAGEFILE || f == ORC),
38+
ZSTD(null, CompressionKind.ZSTD, CompressionCodecName.ZSTD, f -> f == ORC || f == DWRF || f == PAGEFILE || f == PARQUET);
3739

3840
private final Optional<Class<? extends CompressionCodec>> codec;
3941
private final CompressionKind orcCompressionKind;
40-
private final Optional<CompressionCodecName> parquetCompressionCodec;
42+
private final CompressionCodecName parquetCompressionCodec;
4143
private final Predicate<HiveStorageFormat> supportedStorageFormats;
4244

4345
HiveCompressionCodec(
@@ -48,7 +50,7 @@ public enum HiveCompressionCodec
4850
{
4951
this.codec = Optional.ofNullable(codec);
5052
this.orcCompressionKind = requireNonNull(orcCompressionKind, "orcCompressionKind is null");
51-
this.parquetCompressionCodec = Optional.ofNullable(parquetCompressionCodec);
53+
this.parquetCompressionCodec = requireNonNull(parquetCompressionCodec, "parquetCompressionCodec is null");
5254
this.supportedStorageFormats = requireNonNull(supportedStorageFormats, "supportedStorageFormats is null");
5355
}
5456

@@ -62,7 +64,7 @@ public CompressionKind getOrcCompressionKind()
6264
return orcCompressionKind;
6365
}
6466

65-
public Optional<CompressionCodecName> getParquetCompressionCodec()
67+
public CompressionCodecName getParquetCompressionCodec()
6668
{
6769
return parquetCompressionCodec;
6870
}

presto-hive/src/main/java/com/facebook/presto/hive/HiveSessionProperties.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -59,7 +59,7 @@ public final class HiveSessionProperties
5959
private static final String ORC_OPTIMIZED_WRITER_COMPRESSION_LEVEL = "orc_optimized_writer_compression_level";
6060
private static final String PAGEFILE_WRITER_MAX_STRIPE_SIZE = "pagefile_writer_max_stripe_size";
6161
public static final String HIVE_STORAGE_FORMAT = "hive_storage_format";
62-
private static final String COMPRESSION_CODEC = "compression_codec";
62+
static final String COMPRESSION_CODEC = "compression_codec";
6363
private static final String ORC_COMPRESSION_CODEC = "orc_compression_codec";
6464
public static final String RESPECT_TABLE_FORMAT = "respect_table_format";
6565
private static final String CREATE_EMPTY_BUCKET_FILES = "create_empty_bucket_files";

presto-hive/src/main/java/com/facebook/presto/hive/util/ConfigurationUtils.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -120,7 +120,7 @@ private static void setCompressionProperties(Configuration config, HiveCompressi
120120
config.unset(FileOutputFormat.COMPRESS_CODEC);
121121
}
122122
// For Parquet
123-
compression.getParquetCompressionCodec().ifPresent(codec -> config.set(ParquetOutputFormat.COMPRESSION, codec.name()));
123+
config.set(ParquetOutputFormat.COMPRESSION, compression.getParquetCompressionCodec().name());
124124
// For SequenceFile
125125
config.set(FileOutputFormat.COMPRESS_TYPE, BLOCK.toString());
126126
// For PageFile

presto-hive/src/test/java/com/facebook/presto/hive/TestHiveIntegrationSmokeTest.java

Lines changed: 30 additions & 34 deletions
Original file line numberDiff line numberDiff line change
@@ -74,6 +74,7 @@
7474
import java.util.function.BiConsumer;
7575
import java.util.function.Consumer;
7676
import java.util.stream.LongStream;
77+
import java.util.stream.Stream;
7778

7879
import static com.facebook.airlift.json.JsonCodec.jsonCodec;
7980
import static com.facebook.presto.SystemSessionProperties.COLOCATED_JOIN;
@@ -106,6 +107,7 @@
106107
import static com.facebook.presto.hive.HiveQueryRunner.TPCH_SCHEMA;
107108
import static com.facebook.presto.hive.HiveQueryRunner.createBucketedSession;
108109
import static com.facebook.presto.hive.HiveQueryRunner.createMaterializeExchangesSession;
110+
import static com.facebook.presto.hive.HiveSessionProperties.COMPRESSION_CODEC;
109111
import static com.facebook.presto.hive.HiveSessionProperties.FILE_RENAMING_ENABLED;
110112
import static com.facebook.presto.hive.HiveSessionProperties.MANIFEST_VERIFICATION_ENABLED;
111113
import static com.facebook.presto.hive.HiveSessionProperties.OPTIMIZED_PARTITION_UPDATE_SERIALIZATION_ENABLED;
@@ -159,6 +161,7 @@
159161
import static io.airlift.tpch.TpchTable.PART_SUPPLIER;
160162
import static java.lang.String.format;
161163
import static java.nio.charset.StandardCharsets.UTF_8;
164+
import static java.util.Locale.ROOT;
162165
import static java.util.Objects.requireNonNull;
163166
import static java.util.stream.Collectors.joining;
164167
import static org.assertj.core.api.Assertions.assertThat;
@@ -5267,17 +5270,6 @@ public void testPageFileFormatSmallSplitSize()
52675270
assertUpdate("DROP TABLE test_pagefile_small_split");
52685271
}
52695272

5270-
@Test
5271-
public void testPageFileCompression()
5272-
{
5273-
for (HiveCompressionCodec compression : HiveCompressionCodec.values()) {
5274-
if (!compression.isSupportedStorageFormat(PAGEFILE)) {
5275-
continue;
5276-
}
5277-
testPageFileCompression(compression.name());
5278-
}
5279-
}
5280-
52815273
@Test
52825274
public void testPartialAggregatePushdownORC()
52835275
{
@@ -5703,31 +5695,35 @@ public void testParquetSelectivePageSourceFails()
57035695
assertQueryFails(parquetFilterPushdownSession, "SELECT a FROM test_parquet_filter_pushdoown WHERE b = false", "Parquet reader doesn't support filter pushdown yet");
57045696
}
57055697

5706-
private void testPageFileCompression(String compression)
5698+
@DataProvider(name = "testFormatAndCompressionCodecs")
5699+
public Object[][] compressionCodecs()
57075700
{
5708-
Session testSession = Session.builder(getQueryRunner().getDefaultSession())
5709-
.setCatalogSessionProperty(catalog, "compression_codec", compression)
5710-
.setCatalogSessionProperty(catalog, "pagefile_writer_max_stripe_size", "100B")
5711-
.setCatalogSessionProperty(catalog, "max_split_size", "1kB")
5712-
.setCatalogSessionProperty(catalog, "max_initial_split_size", "1kB")
5713-
.build();
5714-
5715-
assertUpdate(
5716-
testSession,
5717-
"CREATE TABLE test_pagefile_compression\n" +
5718-
"WITH (\n" +
5719-
"format = 'PAGEFILE'\n" +
5720-
") AS\n" +
5721-
"SELECT\n" +
5722-
"*\n" +
5723-
"FROM tpch.orders",
5724-
"SELECT count(*) FROM orders");
5725-
5726-
assertQuery(testSession, "SELECT count(*) FROM test_pagefile_compression", "SELECT count(*) FROM orders");
5727-
5728-
assertQuery(testSession, "SELECT sum(custkey) FROM test_pagefile_compression", "SELECT sum(custkey) FROM orders");
5701+
return Stream.of(PARQUET, ORC, PAGEFILE)
5702+
.flatMap(format -> Arrays.stream(HiveCompressionCodec.values())
5703+
.map(codec -> new Object[] {codec, format}))
5704+
.toArray(Object[][]::new);
5705+
}
57295706

5730-
assertUpdate("DROP TABLE test_pagefile_compression");
5707+
@Test(dataProvider = "testFormatAndCompressionCodecs")
5708+
public void testFormatAndCompressionCodecs(HiveCompressionCodec codec, HiveStorageFormat format)
5709+
{
5710+
String tableName = "test_" + format.name().toLowerCase(ROOT) + "_compression_codec_" + codec.name().toLowerCase(ROOT);
5711+
Session session = Session.builder(getSession())
5712+
.setCatalogSessionProperty("hive", COMPRESSION_CODEC, codec.name()).build();
5713+
if (codec.isSupportedStorageFormat(format == PARQUET ? HiveStorageFormat.PARQUET : HiveStorageFormat.ORC)) {
5714+
assertUpdate(session,
5715+
format("CREATE TABLE %s WITH (format = '%s') AS SELECT * FROM orders",
5716+
tableName, format.name()),
5717+
"SELECT count(*) FROM orders");
5718+
assertQuery(format("SELECT count(*) FROM %s", tableName), "SELECT count(*) FROM orders");
5719+
assertQuery(format("SELECT sum(custkey) FROM %s", tableName), "SELECT sum(custkey) FROM orders");
5720+
assertQuerySucceeds(format("DROP TABLE %s", tableName));
5721+
}
5722+
else {
5723+
assertQueryFails(session, format("CREATE TABLE %s WITH (format = '%s') AS SELECT * FROM orders",
5724+
tableName, format.name()),
5725+
format("%s compression is not supported with %s", codec, format));
5726+
}
57315727
}
57325728

57335729
private static Consumer<Plan> assertTableWriterMergeNodeIsPresent()

0 commit comments

Comments
 (0)